From bd4323c27a369be3b7a3a9785e7b22e10f2c9658 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 08:05:06 -0700 Subject: [PATCH 01/10] use map lookup and clean up CometExecRule --- .../scala/org/apache/comet/CometConf.scala | 16 +++ docs/source/user-guide/latest/configs.md | 1 + .../apache/comet/rules/CometExecRule.scala | 86 +++++-------- .../comet/serde/CometOperatorSerde.scala | 21 +++- .../apache/comet/serde/QueryPlanSerde.scala | 109 ++++++++++------ .../serde/operator/CometLocalTableScan.scala | 48 +++++++ .../serde/{ => operator}/CometProject.scala | 3 +- .../serde/{ => operator}/CometSort.scala | 3 +- .../sql/comet/CometLocalTableScanExec.scala | 119 ++++++++++++++++++ .../apache/spark/sql/comet/operators.scala | 2 +- .../apache/comet/exec/CometExecSuite.scala | 24 ++++ 11 files changed, 328 insertions(+), 104 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/serde/operator/CometLocalTableScan.scala rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometProject.scala (94%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometSort.scala (95%) create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 60fd1940bc..496284ce65 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -79,6 +79,8 @@ object CometConf extends ShimCometConf { val COMET_EXPR_CONFIG_PREFIX: String = s"$COMET_PREFIX.expression"; + val COMET_OPERATOR_CONFIG_PREFIX: String = s"$COMET_PREFIX.operator"; + val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") .category(CATEGORY_EXEC) .doc( @@ -251,6 +253,8 @@ object CometConf extends ShimCometConf { createExecEnabledConfig("window", defaultValue = true) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) + val COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED: ConfigEntry[Boolean] = + createExecEnabledConfig("localTableScan", defaultValue = false) val COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.exec.sortMergeJoinWithJoinFilter.enabled") @@ -745,6 +749,18 @@ object CometConf extends ShimCometConf { s"${CometConf.COMET_EXPR_CONFIG_PREFIX}.${exprClass.getSimpleName}.allowIncompatible" } + def isOperatorAllowIncompat(name: String, conf: SQLConf = SQLConf.get): Boolean = { + getBooleanConf(getOperatorAllowIncompatConfigKey(name), defaultValue = false, conf) + } + + def getOperatorAllowIncompatConfigKey(name: String): String = { + s"${CometConf.COMET_OPERATOR_CONFIG_PREFIX}.$name.allowIncompatible" + } + + def getOperatorAllowIncompatConfigKey(exprClass: Class[_]): String = { + s"${CometConf.COMET_OPERATOR_CONFIG_PREFIX}.${exprClass.getSimpleName}.allowIncompatible" + } + def getBooleanConf(name: String, defaultValue: Boolean, conf: SQLConf): Boolean = { conf.getConfString(name, defaultValue.toString).toLowerCase(Locale.ROOT) == "true" } diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 1cc05dfc78..fd232874ec 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -149,6 +149,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.exec.globalLimit.enabled` | Whether to enable globalLimit by default. | true | | `spark.comet.exec.hashJoin.enabled` | Whether to enable hashJoin by default. | true | | `spark.comet.exec.localLimit.enabled` | Whether to enable localLimit by default. | true | +| `spark.comet.exec.localTableScan.enabled` | Whether to enable localTableScan by default. | false | | `spark.comet.exec.project.enabled` | Whether to enable project by default. | true | | `spark.comet.exec.sort.enabled` | Whether to enable sort by default. | true | | `spark.comet.exec.sortMergeJoin.enabled` | Whether to enable sortMergeJoin by default. | true | 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 873a1c55c9..80dabeebdf 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -271,9 +271,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { }) } - case op: ShuffledHashJoinExec - if CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf) && - op.children.forall(isCometNative) => + case op: ShuffledHashJoinExec => newPlanWithProto( op, CometHashJoinExec( @@ -290,15 +288,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case op: ShuffledHashJoinExec if !CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf) => - withInfo(op, "ShuffleHashJoin is not enabled") - - case op: ShuffledHashJoinExec if !op.children.forall(isCometNative) => - op - - case op: BroadcastHashJoinExec - if CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf) && - op.children.forall(isCometNative) => + case op: BroadcastHashJoinExec => newPlanWithProto( op, CometBroadcastHashJoinExec( @@ -315,9 +305,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case op: SortMergeJoinExec - if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && - op.children.forall(isCometNative) => + case op: SortMergeJoinExec => newPlanWithProto( op, CometSortMergeJoinExec( @@ -333,20 +321,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case op: SortMergeJoinExec - if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && - !op.children.forall(isCometNative) => - op - - case op: SortMergeJoinExec if !CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) => - withInfo(op, "SortMergeJoin is not enabled") - - case op: SortMergeJoinExec if !op.children.forall(isCometNative) => - op - - case c @ CoalesceExec(numPartitions, child) - if CometConf.COMET_EXEC_COALESCE_ENABLED.get(conf) - && isCometNative(child) => + case c @ CoalesceExec(numPartitions, child) => QueryPlanSerde .operator2Proto(c) .map { nativeOp => @@ -355,17 +330,17 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } .getOrElse(c) - case c @ CoalesceExec(_, _) if !CometConf.COMET_EXEC_COALESCE_ENABLED.get(conf) => - withInfo(c, "Coalesce is not enabled") - - case op: CoalesceExec if !op.children.forall(isCometNative) => - op - - case s: TakeOrderedAndProjectExec - if isCometNative(s.child) && CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED - .get(conf) - && isCometShuffleEnabled(conf) && - CometTakeOrderedAndProjectExec.isSupported(s) => + case s: TakeOrderedAndProjectExec => + if (!CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED.get(conf)) { + return withInfo(s, "TakeOrderedAndProject is not enabled") + } + if (!isCometShuffleEnabled(conf)) { + return withInfo(s, "TakeOrderedAndProject requires shuffle to be enabled") + } + if (!CometTakeOrderedAndProjectExec.isSupported(s)) { + // call to isSupported adds fallback reasons + return s + } QueryPlanSerde .operator2Proto(s) .map { nativeOp => @@ -382,15 +357,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } .getOrElse(s) - case s: TakeOrderedAndProjectExec => - val info1 = createMessage( - !CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED.get(conf), - "TakeOrderedAndProject is not enabled") - val info2 = createMessage( - !isCometShuffleEnabled(conf), - "TakeOrderedAndProject requires shuffle to be enabled") - withInfo(s, Seq(info1, info2).flatten.mkString(",")) - case w: WindowExec => newPlanWithProto( w, @@ -404,21 +370,16 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { w.child, SerializedPlan(None))) - case u: UnionExec - if CometConf.COMET_EXEC_UNION_ENABLED.get(conf) && - u.children.forall(isCometNative) => + case u: UnionExec => + if (!CometConf.COMET_EXEC_UNION_ENABLED.get(conf)) { + return withInfo(u, "Union is not enabled") + } newPlanWithProto( u, { val cometOp = CometUnionExec(u, u.output, u.children) CometSinkPlaceHolder(_, u, cometOp) }) - case u: UnionExec if !CometConf.COMET_EXEC_UNION_ENABLED.get(conf) => - withInfo(u, "Union is not enabled") - - case op: UnionExec if !op.children.forall(isCometNative) => - op - // For AQE broadcast stage on a Comet broadcast exchange case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) @@ -534,6 +495,15 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { s } + case op: LocalTableScanExec => + QueryPlanSerde + .operator2Proto(op) + .map { nativeOp => + val cometOp = CometLocalTableScanExec(op, op.rows, op.output) + CometScanWrapper(nativeOp, cometOp) + } + .getOrElse(op) + case op => op match { case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | diff --git a/spark/src/main/scala/org/apache/comet/serde/CometOperatorSerde.scala b/spark/src/main/scala/org/apache/comet/serde/CometOperatorSerde.scala index c6a95ec88a..25ea2c1233 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometOperatorSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/CometOperatorSerde.scala @@ -29,6 +29,22 @@ import org.apache.comet.serde.OperatorOuterClass.Operator */ trait CometOperatorSerde[T <: SparkPlan] { + /** + * Get the optional Comet configuration entry that is used to enable or disable native support + * for this operator. + */ + def enabledConfig: Option[ConfigEntry[Boolean]] + + /** + * Determine the support level of the operator based on its attributes. + * + * @param operator + * The Spark operator. + * @return + * Support level (Compatible, Incompatible, or Unsupported). + */ + def getSupportLevel(operator: T): SupportLevel = Compatible(None) + /** * Convert a Spark operator into a protocol buffer representation that can be passed into native * code. @@ -49,9 +65,4 @@ trait CometOperatorSerde[T <: SparkPlan] { builder: Operator.Builder, childOp: Operator*): Option[OperatorOuterClass.Operator] - /** - * Get the optional Comet configuration entry that is used to enable or disable native support - * for this operator. - */ - def enabledConfig: Option[ConfigEntry[Boolean]] } 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 3e0e837c9c..449c572b9c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -20,7 +20,6 @@ package org.apache.comet.serde import scala.jdk.CollectionConverters._ - import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -38,7 +37,6 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHash import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometScan, withInfo} import org.apache.comet.expressions._ @@ -47,6 +45,7 @@ import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.Types.{DataType => ProtoDataType} import org.apache.comet.serde.Types.DataType._ import org.apache.comet.serde.literals.CometLiteral +import org.apache.comet.serde.operator.{CometLocalTableScan, CometProject, CometSort, CometSortOrder} import org.apache.comet.shims.CometExprShim /** @@ -58,7 +57,19 @@ object QueryPlanSerde extends Logging with CometExprShim { * Mapping of Spark operator class to Comet operator handler. */ private val opSerdeMap: Map[Class[_ <: SparkPlan], CometOperatorSerde[_]] = - Map(classOf[ProjectExec] -> CometProject, classOf[SortExec] -> CometSort) + Map( + classOf[ProjectExec] -> CometProject, + classOf[FilterExec] -> CometFilter, + classOf[LocalTableScanExec] -> CometLocalTableScan, + classOf[LocalLimitExec] -> CometLocalLimit, + classOf[GlobalLimitExec] -> CometGlobalLimit, + classOf[ExpandExec] -> CometExpand, + classOf[HashAggregateExec] -> CometHashAggregate, + classOf[ObjectHashAggregateExec] -> CometObjectHashAggregate, + classOf[BroadcastHashJoinExec] -> CometBroadcastHashJoin, + classOf[ShuffledHashJoinExec] -> CometShuffleHashJoin, + classOf[SortMergeJoinExec] -> CometSortMergeJoin, + classOf[SortExec] -> CometSort) private val arrayExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[ArrayAppend] -> CometArrayAppend, @@ -920,51 +931,30 @@ object QueryPlanSerde extends Logging with CometExprShim { val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(op.id) childOp.foreach(builder.addChildren) + // look for registered handler first + val serde = opSerdeMap.get(op.getClass) + serde match { + case Some(handler) if isOperatorEnabled(handler, op) => + val opSerde = handler.asInstanceOf[CometOperatorSerde[SparkPlan]] + val maybeConverted = opSerde.convert(op, builder, childOp: _*) + if (maybeConverted.isDefined) { + return maybeConverted + } + case _ => + } + + // now handle special cases that cannot be handled as a simple mapping from class name + // and see if operator can be used as a sink op match { // Fully native scan for V1 case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => CometNativeScan.convert(scan, builder, childOp: _*) - case filter: FilterExec if CometConf.COMET_EXEC_FILTER_ENABLED.get(conf) => - CometFilter.convert(filter, builder, childOp: _*) - - case limit: LocalLimitExec if CometConf.COMET_EXEC_LOCAL_LIMIT_ENABLED.get(conf) => - CometLocalLimit.convert(limit, builder, childOp: _*) - - case globalLimitExec: GlobalLimitExec - if CometConf.COMET_EXEC_GLOBAL_LIMIT_ENABLED.get(conf) => - CometGlobalLimit.convert(globalLimitExec, builder, childOp: _*) - - case expand: ExpandExec if CometConf.COMET_EXEC_EXPAND_ENABLED.get(conf) => - CometExpand.convert(expand, builder, childOp: _*) - case _: WindowExec if CometConf.COMET_EXEC_WINDOW_ENABLED.get(conf) => withInfo(op, "Window expressions are not supported") None - case aggregate: HashAggregateExec if CometConf.COMET_EXEC_AGGREGATE_ENABLED.get(conf) => - CometHashAggregate.convert(aggregate, builder, childOp: _*) - - case aggregate: ObjectHashAggregateExec - if CometConf.COMET_EXEC_AGGREGATE_ENABLED.get(conf) => - CometObjectHashAggregate.convert(aggregate, builder, childOp: _*) - - case join: BroadcastHashJoinExec - if CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf) => - CometBroadcastHashJoin.convert(join, builder, childOp: _*) - - case join: ShuffledHashJoinExec if CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf) => - CometShuffleHashJoin.convert(join, builder, childOp: _*) - - case join: SortMergeJoinExec => - if (CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf)) { - CometSortMergeJoin.convert(join, builder, childOp: _*) - } else { - withInfo(join, "SortMergeJoin is not enabled") - None - } - case op if isCometSink(op) => val supportedTypes = op.output.forall(a => supportedDataType(a.dataType, allowComplex = true)) @@ -1044,6 +1034,49 @@ object QueryPlanSerde extends Logging with CometExprShim { } } + private def isOperatorEnabled(handler: CometOperatorSerde[_], op: SparkPlan): Boolean = { + val enabled = handler.enabledConfig.forall(_.get(op.conf)) + val opName = op.getClass.getSimpleName + if (enabled) { + val opSerde = handler.asInstanceOf[CometOperatorSerde[SparkPlan]] + opSerde.getSupportLevel(op) match { + case Unsupported(notes) => + withInfo(op, notes.getOrElse("")) + false + case Incompatible(notes) => + val allowIncompat = CometConf.isOperatorAllowIncompat(opName) + val incompatConf = CometConf.getOperatorAllowIncompatConfigKey(opName) + if (allowIncompat) { + if (notes.isDefined) { + logWarning( + s"Comet supports $opName when $incompatConf=true " + + s"but has notes: ${notes.get}") + } + true + } else { + val optionalNotes = notes.map(str => s" ($str)").getOrElse("") + withInfo( + op, + s"$opName is not fully compatible with Spark$optionalNotes. " + + s"To enable it anyway, set $incompatConf=true. " + + s"${CometConf.COMPAT_GUIDE}.") + false + } + case Compatible(notes) => + if (notes.isDefined) { + logWarning(s"Comet supports $opName but has notes: ${notes.get}") + } + true + } + } else { + withInfo( + op, + s"Native support for operator $opName is disabled. " + + s"Set ${handler.enabledConfig.get.key}=true to enable it.") + false + } + } + /** * Whether the input Spark operator `op` can be considered as a Comet sink, i.e., the start of * native execution. If it is true, we'll wrap `op` with `CometScanWrapper` or diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometLocalTableScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometLocalTableScan.scala new file mode 100644 index 0000000000..e3e8538cf6 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometLocalTableScan.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.serde.operator + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.execution.LocalTableScanExec + +import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde.serializeDataType + +object CometLocalTableScan extends CometOperatorSerde[LocalTableScanExec] { + + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED) + + override def convert( + op: LocalTableScanExec, + builder: Operator.Builder, + childOp: Operator*): Option[Operator] = { + val scanTypes = op.output.flatten(attr => serializeDataType(attr.dataType)) + val scanBuilder = OperatorOuterClass.Scan + .newBuilder() + .setSource(op.getClass.getSimpleName) + .addAllFields(scanTypes.asJava) + .setArrowFfiSafe(false) + Some(builder.setScan(scanBuilder).build()) + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/CometProject.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometProject.scala similarity index 94% rename from spark/src/main/scala/org/apache/comet/serde/CometProject.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometProject.scala index 651aa8fefd..4ba02945d6 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometProject.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometProject.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ @@ -25,6 +25,7 @@ import org.apache.spark.sql.execution.ProjectExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.exprToProto diff --git a/spark/src/main/scala/org/apache/comet/serde/CometSort.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometSort.scala similarity index 95% rename from spark/src/main/scala/org/apache/comet/serde/CometSort.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometSort.scala index 4a1063458c..39a1c55656 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometSort.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometSort.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ @@ -27,6 +27,7 @@ import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, MapType, Stru import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometExpressionSerde, CometOperatorSerde, Compatible, ExprOuterClass, Incompatible, OperatorOuterClass, SupportLevel} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{exprToProto, exprToProtoInternal, supportedSortType} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala new file mode 100644 index 0000000000..611d367f35 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometLocalTableScanExec.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} +import org.apache.spark.sql.comet.CometLocalTableScanExec.createMetricsIterator +import org.apache.spark.sql.comet.execution.arrow.CometArrowConverters +import org.apache.spark.sql.execution.{LeafExecNode, LocalTableScanExec} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import com.google.common.base.Objects + +import org.apache.comet.CometConf + +case class CometLocalTableScanExec( + originalPlan: LocalTableScanExec, + @transient rows: Seq[InternalRow], + override val output: Seq[Attribute]) + extends CometExec + with LeafExecNode { + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + @transient private lazy val unsafeRows: Array[InternalRow] = { + if (rows.isEmpty) { + Array.empty + } else { + val proj = UnsafeProjection.create(output, output) + rows.map(r => proj(r).copy()).toArray + } + } + + @transient private lazy val rdd: RDD[InternalRow] = { + if (rows.isEmpty) { + sparkContext.emptyRDD + } else { + val numSlices = math.min(unsafeRows.length, session.leafNodeDefaultParallelism) + sparkContext.parallelize(unsafeRows, numSlices) + } + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numInputRows = longMetric("numOutputRows") + val maxRecordsPerBatch = CometConf.COMET_BATCH_SIZE.get(conf) + val timeZoneId = conf.sessionLocalTimeZone + rdd.mapPartitionsInternal { sparkBatches => + val context = TaskContext.get() + val batches = CometArrowConverters.rowToArrowBatchIter( + sparkBatches, + originalPlan.schema, + maxRecordsPerBatch, + timeZoneId, + context) + createMetricsIterator(batches, numInputRows) + } + } + + override protected def stringArgs: Iterator[Any] = { + if (rows.isEmpty) { + Iterator("", output) + } else { + Iterator(output) + } + } + + override def supportsColumnar: Boolean = true + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometLocalTableScanExec => + this.originalPlan == other.originalPlan && + this.schema == other.schema && + this.output == other.output + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(originalPlan, originalPlan.schema, output) +} + +object CometLocalTableScanExec { + + private def createMetricsIterator( + it: Iterator[ColumnarBatch], + numInputRows: SQLMetric): Iterator[ColumnarBatch] = { + new Iterator[ColumnarBatch] { + override def hasNext: Boolean = it.hasNext + + override def next(): ColumnarBatch = { + val batch = it.next() + numInputRows.add(batch.numRows()) + batch + } + } + } +} 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 de6892638a..d7a743eb2d 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 @@ -380,7 +380,7 @@ abstract class CometNativeExec extends CometExec { _: ShuffleQueryStageExec | _: AQEShuffleReadExec | _: CometShuffleExchangeExec | _: CometUnionExec | _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | _: ReusedExchangeExec | _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec | - _: CometSparkToColumnarExec => + _: CometSparkToColumnarExec | _: CometLocalTableScanExec => func(plan) case _: CometPlan => // Other Comet operators, continue to traverse the tree. diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index fb8af7efbc..aadd1adfd0 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -2105,6 +2105,30 @@ class CometExecSuite extends CometTestBase { } } + test("LocalTableScanExec spark fallback") { + withSQLConf(CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "false") { + val df = Seq.range(0, 10).toDF("id") + checkSparkAnswerAndFallbackReason(df, "LocalTableScan is not enabled") + } + } + + test("LocalTableScanExec with filter") { + withSQLConf(CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val df = Seq.range(0, 10).toDF("id").filter(col("id") > 5) + checkSparkAnswerAndOperator(df) + } + } + + test("LocalTableScanExec with groupBy") { + withSQLConf(CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val df = (Seq.range(0, 10) ++ Seq.range(0, 20)) + .toDF("id") + .groupBy(col("id")) + .agg(count("*")) + checkSparkAnswerAndOperator(df) + } + } + } case class BucketedTableTestSpec( From e9bb02e6b7846e7c420859c4595dc161f59691f8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 08:10:46 -0700 Subject: [PATCH 02/10] move operators into package --- .../apache/comet/serde/QueryPlanSerde.scala | 26 ++---------- .../org/apache/comet/serde/SupportLevel.scala | 42 +++++++++++++++++++ .../serde/{ => operator}/CometAggregate.scala | 3 +- .../serde/{ => operator}/CometExpand.scala | 3 +- .../serde/{ => operator}/CometFilter.scala | 3 +- .../{ => operator}/CometGlobalLimit.scala | 3 +- .../serde/{ => operator}/CometHashJoin.scala | 5 ++- .../{ => operator}/CometLocalLimit.scala | 3 +- .../{ => operator}/CometNativeScan.scala | 3 +- .../{ => operator}/CometSortMergeJoin.scala | 7 ++-- .../serde/{ => operator}/CometWindow.scala | 3 +- 11 files changed, 66 insertions(+), 35 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/serde/SupportLevel.scala rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometAggregate.scala (98%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometExpand.scala (94%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometFilter.scala (94%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometGlobalLimit.scala (93%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometHashJoin.scala (96%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometLocalLimit.scala (94%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometNativeScan.scala (98%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometSortMergeJoin.scala (94%) rename spark/src/main/scala/org/apache/comet/serde/{ => operator}/CometWindow.scala (97%) 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 449c572b9c..4db9fb4743 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -20,6 +20,7 @@ package org.apache.comet.serde import scala.jdk.CollectionConverters._ + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -37,6 +38,7 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHash import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ + import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometScan, withInfo} import org.apache.comet.expressions._ @@ -45,7 +47,7 @@ import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.Types.{DataType => ProtoDataType} import org.apache.comet.serde.Types.DataType._ import org.apache.comet.serde.literals.CometLiteral -import org.apache.comet.serde.operator.{CometLocalTableScan, CometProject, CometSort, CometSortOrder} +import org.apache.comet.serde.operator._ import org.apache.comet.shims.CometExprShim /** @@ -1173,25 +1175,3 @@ object QueryPlanSerde extends Logging with CometExprShim { } } - -sealed trait SupportLevel - -/** - * Comet either supports this feature with full compatibility with Spark, or may have known - * differences in some specific edge cases that are unlikely to be an issue for most users. - * - * Any compatibility differences are noted in the - * [[https://datafusion.apache.org/comet/user-guide/compatibility.html Comet Compatibility Guide]]. - */ -case class Compatible(notes: Option[String] = None) extends SupportLevel - -/** - * Comet supports this feature but results can be different from Spark. - * - * Any compatibility differences are noted in the - * [[https://datafusion.apache.org/comet/user-guide/compatibility.html Comet Compatibility Guide]]. - */ -case class Incompatible(notes: Option[String] = None) extends SupportLevel - -/** Comet does not support this feature */ -case class Unsupported(notes: Option[String] = None) extends SupportLevel diff --git a/spark/src/main/scala/org/apache/comet/serde/SupportLevel.scala b/spark/src/main/scala/org/apache/comet/serde/SupportLevel.scala new file mode 100644 index 0000000000..d5a524077d --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/SupportLevel.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.serde + +sealed trait SupportLevel + +/** + * Comet either supports this feature with full compatibility with Spark, or may have known + * differences in some specific edge cases that are unlikely to be an issue for most users. + * + * Any compatibility differences are noted in the + * [[https://datafusion.apache.org/comet/user-guide/compatibility.html Comet Compatibility Guide]]. + */ +case class Compatible(notes: Option[String] = None) extends SupportLevel + +/** + * Comet supports this feature but results can be different from Spark. + * + * Any compatibility differences are noted in the + * [[https://datafusion.apache.org/comet/user-guide/compatibility.html Comet Compatibility Guide]]. + */ +case class Incompatible(notes: Option[String] = None) extends SupportLevel + +/** Comet does not support this feature */ +case class Unsupported(notes: Option[String] = None) extends SupportLevel diff --git a/spark/src/main/scala/org/apache/comet/serde/CometAggregate.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala similarity index 98% rename from spark/src/main/scala/org/apache/comet/serde/CometAggregate.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala index f0cf244f1e..93e5d52c8d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometAggregate.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ @@ -28,6 +28,7 @@ import org.apache.spark.sql.types.MapType import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto} diff --git a/spark/src/main/scala/org/apache/comet/serde/CometExpand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometExpand.scala similarity index 94% rename from spark/src/main/scala/org/apache/comet/serde/CometExpand.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometExpand.scala index 5979eed4dc..ab5a58b064 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometExpand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometExpand.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.ExpandExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.exprToProto diff --git a/spark/src/main/scala/org/apache/comet/serde/CometFilter.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometFilter.scala similarity index 94% rename from spark/src/main/scala/org/apache/comet/serde/CometFilter.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometFilter.scala index 1638750b5f..96771b902f 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometFilter.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometFilter.scala @@ -17,12 +17,13 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import org.apache.spark.sql.execution.FilterExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.exprToProto diff --git a/spark/src/main/scala/org/apache/comet/serde/CometGlobalLimit.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometGlobalLimit.scala similarity index 93% rename from spark/src/main/scala/org/apache/comet/serde/CometGlobalLimit.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometGlobalLimit.scala index 774e1ad77e..b2df3cf72c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometGlobalLimit.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometGlobalLimit.scala @@ -17,12 +17,13 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import org.apache.spark.sql.execution.GlobalLimitExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator object CometGlobalLimit extends CometOperatorSerde[GlobalLimitExec] { diff --git a/spark/src/main/scala/org/apache/comet/serde/CometHashJoin.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometHashJoin.scala similarity index 96% rename from spark/src/main/scala/org/apache/comet/serde/CometHashJoin.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometHashJoin.scala index 67fb67a2e7..c58384e3a9 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometHashJoin.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometHashJoin.scala @@ -17,17 +17,18 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} -import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftAnti, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec} import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.{BuildSide, JoinType, Operator} import org.apache.comet.serde.QueryPlanSerde.exprToProto diff --git a/spark/src/main/scala/org/apache/comet/serde/CometLocalLimit.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometLocalLimit.scala similarity index 94% rename from spark/src/main/scala/org/apache/comet/serde/CometLocalLimit.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometLocalLimit.scala index 1347b12907..3e5fbdebb3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometLocalLimit.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometLocalLimit.scala @@ -17,12 +17,13 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import org.apache.spark.sql.execution.LocalLimitExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator object CometLocalLimit extends CometOperatorSerde[LocalLimitExec] { diff --git a/spark/src/main/scala/org/apache/comet/serde/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala similarity index 98% rename from spark/src/main/scala/org/apache/comet/serde/CometNativeScan.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index 476313a9d1..2bc8b5526a 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ @@ -35,6 +35,7 @@ import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.CometParquetUtils +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.ExprOuterClass.Expr import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} diff --git a/spark/src/main/scala/org/apache/comet/serde/CometSortMergeJoin.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometSortMergeJoin.scala similarity index 94% rename from spark/src/main/scala/org/apache/comet/serde/CometSortMergeJoin.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometSortMergeJoin.scala index 5f926f06e8..a22230d7aa 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometSortMergeJoin.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometSortMergeJoin.scala @@ -17,17 +17,18 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.expressions.{Ascending, Expression, ExpressionSet, SortOrder} -import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftAnti, LeftOuter, LeftSemi, RightOuter} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.SortMergeJoinExec -import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, TimestampNTZType} +import org.apache.spark.sql.types._ import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.{JoinType, Operator} import org.apache.comet.serde.QueryPlanSerde.exprToProto diff --git a/spark/src/main/scala/org/apache/comet/serde/CometWindow.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometWindow.scala similarity index 97% rename from spark/src/main/scala/org/apache/comet/serde/CometWindow.scala rename to spark/src/main/scala/org/apache/comet/serde/operator/CometWindow.scala index 7e963d6326..8f560bed5b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometWindow.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometWindow.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.serde +package org.apache.comet.serde.operator import scala.jdk.CollectionConverters._ @@ -27,6 +27,7 @@ import org.apache.spark.sql.execution.window.WindowExec import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{exprToProto, windowExprToProto} From 9bd17e265e96de3c79b980e4eea2ed4a9b55d46c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 08:27:09 -0700 Subject: [PATCH 03/10] more cleanup --- .../apache/comet/rules/CometExecRule.scala | 117 ++++++++++-------- .../comet/serde/operator/CometAggregate.scala | 7 +- 2 files changed, 69 insertions(+), 55 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 80dabeebdf..1bc5c98e5b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -155,6 +155,41 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { operator2Proto(op).map(fun).getOrElse(op) } + /** Common code for HashAggregateExec and ObjectHashAggregateExec */ + def convertAggregate(op: BaseAggregateExec): SparkPlan = { + val modes = op.aggregateExpressions.map(_.mode).distinct + // In distinct aggregates there can be a combination of modes + val multiMode = modes.size > 1 + // For a final mode HashAggregate, we only need to transform the HashAggregate + // if there is Comet partial aggregation. + val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(op.child).isEmpty + + if (multiMode || sparkFinalMode) { + op + } else { + newPlanWithProto( + op, + nativeOp => { + // The aggExprs could be empty. For example, if the aggregate functions only have + // distinct aggregate functions or only have group by, the aggExprs is empty and + // modes is empty too. If aggExprs is not empty, we need to verify all the + // aggregates have the same mode. + assert(modes.length == 1 || modes.isEmpty) + CometHashAggregateExec( + nativeOp, + op, + op.output, + op.groupingExpressions, + op.aggregateExpressions, + op.resultExpressions, + op.child.output, + modes.headOption, + op.child, + SerializedPlan(None)) + }) + } + } + def convertNode(op: SparkPlan): SparkPlan = op match { // Fully native scan for V1 case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => @@ -235,41 +270,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // When Comet shuffle is disabled, we don't want to transform the HashAggregate // to CometHashAggregate. Otherwise, we probably get partial Comet aggregation // and final Spark aggregation. - case op: BaseAggregateExec - if op.isInstanceOf[HashAggregateExec] || - op.isInstanceOf[ObjectHashAggregateExec] && - isCometShuffleEnabled(conf) => - val modes = op.aggregateExpressions.map(_.mode).distinct - // In distinct aggregates there can be a combination of modes - val multiMode = modes.size > 1 - // For a final mode HashAggregate, we only need to transform the HashAggregate - // if there is Comet partial aggregation. - val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(op.child).isEmpty - - if (multiMode || sparkFinalMode) { - op - } else { - newPlanWithProto( - op, - nativeOp => { - // The aggExprs could be empty. For example, if the aggregate functions only have - // distinct aggregate functions or only have group by, the aggExprs is empty and - // modes is empty too. If aggExprs is not empty, we need to verify all the - // aggregates have the same mode. - assert(modes.length == 1 || modes.isEmpty) - CometHashAggregateExec( - nativeOp, - op, - op.output, - op.groupingExpressions, - op.aggregateExpressions, - op.resultExpressions, - op.child.output, - modes.headOption, - op.child, - SerializedPlan(None)) - }) - } + case op: HashAggregateExec => + convertAggregate(op) + + case op: ObjectHashAggregateExec => + convertAggregate(op) case op: ShuffledHashJoinExec => newPlanWithProto( @@ -390,11 +395,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { _) => newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast - // exchange. It is only used for Comet native execution. We only transform Spark broadcast - // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the - // broadcast exchange is forced to be enabled by Comet config. case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => + // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast + // exchange. It is only used for Comet native execution. We only transform Spark broadcast + // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the + // broadcast exchange is forced to be enabled by Comet config. val newChildren = plan.children.map { case b: BroadcastExchangeExec if isCometNative(b.child) && @@ -407,7 +412,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } case other => other } - if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { + val newPlan = if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { val newPlan = convertNode(plan.withNewChildren(newChildren)) if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { newPlan @@ -423,15 +428,20 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { plan } - // this case should be checked only after the previous case checking for a - // child BroadcastExchange has been applied, otherwise that transform - // never gets applied - case op: BroadcastHashJoinExec if !op.children.forall(isCometNative) => - op - - case op: BroadcastHashJoinExec - if !CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf) => - withInfo(op, "BroadcastHashJoin is not enabled") + // this case should be checked only after the previous case checking for a + // child BroadcastExchange has been applied, otherwise that transform + // never gets applied + if (newPlan.children.forall(isCometNative)) { + // it seems odd to check this last, but that is how the original code was + // implemented + if (CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf)) { + newPlan + } else { + withInfo(op, "BroadcastHashJoin is not enabled") + } + } else { + op + } // For AQE shuffle stage on a Comet shuffle exchange case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => @@ -496,13 +506,12 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } case op: LocalTableScanExec => - QueryPlanSerde - .operator2Proto(op) - .map { nativeOp => + newPlanWithProto( + op, + { nativeOp => val cometOp = CometLocalTableScanExec(op, op.rows, op.output) CometScanWrapper(nativeOp, cometOp) - } - .getOrElse(op) + }) case op => op match { diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala index 93e5d52c8d..4654b722ee 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregat import org.apache.spark.sql.types.MapType import org.apache.comet.{CometConf, ConfigEntry} -import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.CometSparkSessionExtensions.{isCometShuffleEnabled, withInfo} import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto} @@ -44,6 +44,11 @@ trait CometBaseAggregate { val resultExpressions = aggregate.resultExpressions val child = aggregate.child + if (!isCometShuffleEnabled(aggregate.conf)) { + withInfo(aggregate, "Aggregates are only supported when Comet shuffle is enabled") + return None + } + if (groupingExpressions.isEmpty && aggregateExpressions.isEmpty) { withInfo(aggregate, "No group by or aggregation") return None From 7c64c4be1638f66a68567c7f173cb448fd107962 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 08:36:42 -0700 Subject: [PATCH 04/10] simplify --- .../main/scala/org/apache/comet/rules/CometExecRule.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 1bc5c98e5b..ccee4f7c6c 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -506,12 +506,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } case op: LocalTableScanExec => - newPlanWithProto( - op, - { nativeOp => - val cometOp = CometLocalTableScanExec(op, op.rows, op.output) - CometScanWrapper(nativeOp, cometOp) - }) + newPlanWithProto(op, CometScanWrapper(_, CometLocalTableScanExec(op, op.rows, op.output))) case op => op match { From de1a058f18bdfeb2c8601a74fdee2fc8d137072f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 08:56:55 -0700 Subject: [PATCH 05/10] revert CometExecRule changes --- .../apache/comet/rules/CometExecRule.scala | 188 +- .../q10a.native_iceberg_compat/explain.txt | 368 +- .../q10a.native_iceberg_compat/extended.txt | 117 +- .../q10a.native_iceberg_compat/simplified.txt | 124 +- .../q10a/explain.txt | 368 +- .../q10a/extended.txt | 117 +- .../q10a/simplified.txt | 124 +- .../q11.native_iceberg_compat/explain.txt | 745 +-- .../q11.native_iceberg_compat/extended.txt | 191 +- .../q11.native_iceberg_compat/simplified.txt | 214 +- .../q11/explain.txt | 745 +-- .../q11/extended.txt | 191 +- .../q11/simplified.txt | 214 +- .../q12.native_iceberg_compat/explain.txt | 191 +- .../q12.native_iceberg_compat/extended.txt | 66 +- .../q12.native_iceberg_compat/simplified.txt | 78 +- .../q12/explain.txt | 191 +- .../q12/extended.txt | 66 +- .../q12/simplified.txt | 78 +- .../q14.native_iceberg_compat/explain.txt | 1098 ++-- .../q14.native_iceberg_compat/extended.txt | 739 +-- .../q14.native_iceberg_compat/simplified.txt | 347 +- .../q14/explain.txt | 1098 ++-- .../q14/extended.txt | 739 +-- .../q14/simplified.txt | 347 +- .../q14a.native_iceberg_compat/explain.txt | 1520 ++--- .../q14a.native_iceberg_compat/extended.txt | 5115 +++++++++-------- .../q14a.native_iceberg_compat/simplified.txt | 447 +- .../q14a/explain.txt | 1520 ++--- .../q14a/extended.txt | 5115 +++++++++-------- .../q14a/simplified.txt | 447 +- .../q18a.native_iceberg_compat/explain.txt | 1380 +++-- .../q18a.native_iceberg_compat/extended.txt | 463 +- .../q18a.native_iceberg_compat/simplified.txt | 395 +- .../q18a/explain.txt | 1380 +++-- .../q18a/extended.txt | 463 +- .../q18a/simplified.txt | 395 +- .../q20.native_iceberg_compat/explain.txt | 191 +- .../q20.native_iceberg_compat/extended.txt | 66 +- .../q20.native_iceberg_compat/simplified.txt | 78 +- .../q20/explain.txt | 191 +- .../q20/extended.txt | 66 +- .../q20/simplified.txt | 78 +- .../q22.native_iceberg_compat/explain.txt | 201 +- .../q22.native_iceberg_compat/extended.txt | 70 +- .../q22.native_iceberg_compat/simplified.txt | 86 +- .../q22/explain.txt | 201 +- .../q22/extended.txt | 70 +- .../q22/simplified.txt | 86 +- .../q22a.native_iceberg_compat/explain.txt | 511 +- .../q22a.native_iceberg_compat/extended.txt | 342 +- .../q22a.native_iceberg_compat/simplified.txt | 141 +- .../q22a/explain.txt | 511 +- .../q22a/extended.txt | 342 +- .../q22a/simplified.txt | 141 +- .../q24.native_iceberg_compat/explain.txt | 400 +- .../q24.native_iceberg_compat/extended.txt | 120 +- .../q24.native_iceberg_compat/simplified.txt | 102 +- .../q24/explain.txt | 400 +- .../q24/extended.txt | 120 +- .../q24/simplified.txt | 102 +- .../q27a.native_iceberg_compat/explain.txt | 672 ++- .../q27a.native_iceberg_compat/extended.txt | 211 +- .../q27a.native_iceberg_compat/simplified.txt | 199 +- .../q27a/explain.txt | 672 ++- .../q27a/extended.txt | 211 +- .../q27a/simplified.txt | 199 +- .../q34.native_iceberg_compat/explain.txt | 267 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q34.native_iceberg_compat/simplified.txt | 92 +- .../q34/explain.txt | 267 +- .../q34/extended.txt | 82 +- .../q34/simplified.txt | 92 +- .../q35.native_iceberg_compat/explain.txt | 365 +- .../q35.native_iceberg_compat/extended.txt | 130 +- .../q35.native_iceberg_compat/simplified.txt | 142 +- .../q35/explain.txt | 365 +- .../q35/extended.txt | 130 +- .../q35/simplified.txt | 142 +- .../q35a.native_iceberg_compat/explain.txt | 372 +- .../q35a.native_iceberg_compat/extended.txt | 117 +- .../q35a.native_iceberg_compat/simplified.txt | 124 +- .../q35a/explain.txt | 372 +- .../q35a/extended.txt | 117 +- .../q35a/simplified.txt | 124 +- .../q36a.native_iceberg_compat/explain.txt | 429 +- .../q36a.native_iceberg_compat/extended.txt | 223 +- .../q36a.native_iceberg_compat/simplified.txt | 134 +- .../q36a/explain.txt | 429 +- .../q36a/extended.txt | 223 +- .../q36a/simplified.txt | 134 +- .../q47.native_iceberg_compat/explain.txt | 377 +- .../q47.native_iceberg_compat/extended.txt | 225 +- .../q47.native_iceberg_compat/simplified.txt | 150 +- .../q47/explain.txt | 377 +- .../q47/extended.txt | 225 +- .../q47/simplified.txt | 150 +- .../q49.native_iceberg_compat/explain.txt | 713 +-- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q49.native_iceberg_compat/simplified.txt | 168 +- .../q49/explain.txt | 713 +-- .../q49/extended.txt | 144 +- .../q49/simplified.txt | 168 +- .../q51a.native_iceberg_compat/explain.txt | 687 +-- .../q51a.native_iceberg_compat/extended.txt | 456 +- .../q51a.native_iceberg_compat/simplified.txt | 229 +- .../q51a/explain.txt | 687 +-- .../q51a/extended.txt | 456 +- .../q51a/simplified.txt | 229 +- .../q57.native_iceberg_compat/explain.txt | 377 +- .../q57.native_iceberg_compat/extended.txt | 225 +- .../q57.native_iceberg_compat/simplified.txt | 150 +- .../q57/explain.txt | 377 +- .../q57/extended.txt | 225 +- .../q57/simplified.txt | 150 +- .../q5a.native_iceberg_compat/explain.txt | 810 +-- .../q5a.native_iceberg_compat/extended.txt | 571 +- .../q5a.native_iceberg_compat/simplified.txt | 248 +- .../q5a/explain.txt | 810 +-- .../q5a/extended.txt | 571 +- .../q5a/simplified.txt | 248 +- .../q6.native_iceberg_compat/explain.txt | 392 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q6.native_iceberg_compat/simplified.txt | 132 +- .../q6/explain.txt | 392 +- .../q6/extended.txt | 123 +- .../q6/simplified.txt | 132 +- .../q64.native_iceberg_compat/explain.txt | 1426 ++--- .../q64.native_iceberg_compat/extended.txt | 514 +- .../q64.native_iceberg_compat/simplified.txt | 441 +- .../q64/explain.txt | 1426 ++--- .../q64/extended.txt | 514 +- .../q64/simplified.txt | 441 +- .../q67a.native_iceberg_compat/explain.txt | 849 +-- .../q67a.native_iceberg_compat/extended.txt | 648 ++- .../q67a.native_iceberg_compat/simplified.txt | 239 +- .../q67a/explain.txt | 849 +-- .../q67a/extended.txt | 648 ++- .../q67a/simplified.txt | 239 +- .../q70a.native_iceberg_compat/explain.txt | 622 +- .../q70a.native_iceberg_compat/extended.txt | 362 +- .../q70a.native_iceberg_compat/simplified.txt | 206 +- .../q70a/explain.txt | 622 +- .../q70a/extended.txt | 362 +- .../q70a/simplified.txt | 206 +- .../q72.native_iceberg_compat/explain.txt | 492 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q72.native_iceberg_compat/simplified.txt | 155 +- .../q72/explain.txt | 492 +- .../q72/extended.txt | 126 +- .../q72/simplified.txt | 155 +- .../q74.native_iceberg_compat/explain.txt | 745 +-- .../q74.native_iceberg_compat/extended.txt | 191 +- .../q74.native_iceberg_compat/simplified.txt | 214 +- .../q74/explain.txt | 745 +-- .../q74/extended.txt | 191 +- .../q74/simplified.txt | 214 +- .../q75.native_iceberg_compat/explain.txt | 680 +-- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q75.native_iceberg_compat/simplified.txt | 207 +- .../q75/explain.txt | 680 +-- .../q75/extended.txt | 216 +- .../q75/simplified.txt | 207 +- .../q77a.native_iceberg_compat/explain.txt | 1019 ++-- .../q77a.native_iceberg_compat/extended.txt | 686 ++- .../q77a.native_iceberg_compat/simplified.txt | 251 +- .../q77a/explain.txt | 1019 ++-- .../q77a/extended.txt | 686 ++- .../q77a/simplified.txt | 251 +- .../q78.native_iceberg_compat/explain.txt | 525 +- .../q78.native_iceberg_compat/extended.txt | 168 +- .../q78.native_iceberg_compat/simplified.txt | 178 +- .../q78/explain.txt | 525 +- .../q78/extended.txt | 168 +- .../q78/simplified.txt | 178 +- .../q80a.native_iceberg_compat/explain.txt | 1069 ++-- .../q80a.native_iceberg_compat/extended.txt | 829 +-- .../q80a.native_iceberg_compat/simplified.txt | 309 +- .../q80a/explain.txt | 1069 ++-- .../q80a/extended.txt | 829 +-- .../q80a/simplified.txt | 309 +- .../q86a.native_iceberg_compat/explain.txt | 376 +- .../q86a.native_iceberg_compat/extended.txt | 184 +- .../q86a.native_iceberg_compat/simplified.txt | 122 +- .../q86a/explain.txt | 376 +- .../q86a/extended.txt | 184 +- .../q86a/simplified.txt | 122 +- .../q98.native_iceberg_compat/explain.txt | 159 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q98.native_iceberg_compat/simplified.txt | 66 +- .../q98/explain.txt | 159 +- .../q98/extended.txt | 54 +- .../q98/simplified.txt | 66 +- 193 files changed, 43467 insertions(+), 36661 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index ccee4f7c6c..873a1c55c9 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -155,41 +155,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { operator2Proto(op).map(fun).getOrElse(op) } - /** Common code for HashAggregateExec and ObjectHashAggregateExec */ - def convertAggregate(op: BaseAggregateExec): SparkPlan = { - val modes = op.aggregateExpressions.map(_.mode).distinct - // In distinct aggregates there can be a combination of modes - val multiMode = modes.size > 1 - // For a final mode HashAggregate, we only need to transform the HashAggregate - // if there is Comet partial aggregation. - val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(op.child).isEmpty - - if (multiMode || sparkFinalMode) { - op - } else { - newPlanWithProto( - op, - nativeOp => { - // The aggExprs could be empty. For example, if the aggregate functions only have - // distinct aggregate functions or only have group by, the aggExprs is empty and - // modes is empty too. If aggExprs is not empty, we need to verify all the - // aggregates have the same mode. - assert(modes.length == 1 || modes.isEmpty) - CometHashAggregateExec( - nativeOp, - op, - op.output, - op.groupingExpressions, - op.aggregateExpressions, - op.resultExpressions, - op.child.output, - modes.headOption, - op.child, - SerializedPlan(None)) - }) - } - } - def convertNode(op: SparkPlan): SparkPlan = op match { // Fully native scan for V1 case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => @@ -270,13 +235,45 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // When Comet shuffle is disabled, we don't want to transform the HashAggregate // to CometHashAggregate. Otherwise, we probably get partial Comet aggregation // and final Spark aggregation. - case op: HashAggregateExec => - convertAggregate(op) - - case op: ObjectHashAggregateExec => - convertAggregate(op) + case op: BaseAggregateExec + if op.isInstanceOf[HashAggregateExec] || + op.isInstanceOf[ObjectHashAggregateExec] && + isCometShuffleEnabled(conf) => + val modes = op.aggregateExpressions.map(_.mode).distinct + // In distinct aggregates there can be a combination of modes + val multiMode = modes.size > 1 + // For a final mode HashAggregate, we only need to transform the HashAggregate + // if there is Comet partial aggregation. + val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(op.child).isEmpty + + if (multiMode || sparkFinalMode) { + op + } else { + newPlanWithProto( + op, + nativeOp => { + // The aggExprs could be empty. For example, if the aggregate functions only have + // distinct aggregate functions or only have group by, the aggExprs is empty and + // modes is empty too. If aggExprs is not empty, we need to verify all the + // aggregates have the same mode. + assert(modes.length == 1 || modes.isEmpty) + CometHashAggregateExec( + nativeOp, + op, + op.output, + op.groupingExpressions, + op.aggregateExpressions, + op.resultExpressions, + op.child.output, + modes.headOption, + op.child, + SerializedPlan(None)) + }) + } - case op: ShuffledHashJoinExec => + case op: ShuffledHashJoinExec + if CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf) && + op.children.forall(isCometNative) => newPlanWithProto( op, CometHashJoinExec( @@ -293,7 +290,15 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case op: BroadcastHashJoinExec => + case op: ShuffledHashJoinExec if !CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf) => + withInfo(op, "ShuffleHashJoin is not enabled") + + case op: ShuffledHashJoinExec if !op.children.forall(isCometNative) => + op + + case op: BroadcastHashJoinExec + if CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf) && + op.children.forall(isCometNative) => newPlanWithProto( op, CometBroadcastHashJoinExec( @@ -310,7 +315,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case op: SortMergeJoinExec => + case op: SortMergeJoinExec + if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && + op.children.forall(isCometNative) => newPlanWithProto( op, CometSortMergeJoinExec( @@ -326,7 +333,20 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { op.right, SerializedPlan(None))) - case c @ CoalesceExec(numPartitions, child) => + case op: SortMergeJoinExec + if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && + !op.children.forall(isCometNative) => + op + + case op: SortMergeJoinExec if !CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) => + withInfo(op, "SortMergeJoin is not enabled") + + case op: SortMergeJoinExec if !op.children.forall(isCometNative) => + op + + case c @ CoalesceExec(numPartitions, child) + if CometConf.COMET_EXEC_COALESCE_ENABLED.get(conf) + && isCometNative(child) => QueryPlanSerde .operator2Proto(c) .map { nativeOp => @@ -335,17 +355,17 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } .getOrElse(c) - case s: TakeOrderedAndProjectExec => - if (!CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED.get(conf)) { - return withInfo(s, "TakeOrderedAndProject is not enabled") - } - if (!isCometShuffleEnabled(conf)) { - return withInfo(s, "TakeOrderedAndProject requires shuffle to be enabled") - } - if (!CometTakeOrderedAndProjectExec.isSupported(s)) { - // call to isSupported adds fallback reasons - return s - } + case c @ CoalesceExec(_, _) if !CometConf.COMET_EXEC_COALESCE_ENABLED.get(conf) => + withInfo(c, "Coalesce is not enabled") + + case op: CoalesceExec if !op.children.forall(isCometNative) => + op + + case s: TakeOrderedAndProjectExec + if isCometNative(s.child) && CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED + .get(conf) + && isCometShuffleEnabled(conf) && + CometTakeOrderedAndProjectExec.isSupported(s) => QueryPlanSerde .operator2Proto(s) .map { nativeOp => @@ -362,6 +382,15 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } .getOrElse(s) + case s: TakeOrderedAndProjectExec => + val info1 = createMessage( + !CometConf.COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED.get(conf), + "TakeOrderedAndProject is not enabled") + val info2 = createMessage( + !isCometShuffleEnabled(conf), + "TakeOrderedAndProject requires shuffle to be enabled") + withInfo(s, Seq(info1, info2).flatten.mkString(",")) + case w: WindowExec => newPlanWithProto( w, @@ -375,16 +404,21 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { w.child, SerializedPlan(None))) - case u: UnionExec => - if (!CometConf.COMET_EXEC_UNION_ENABLED.get(conf)) { - return withInfo(u, "Union is not enabled") - } + case u: UnionExec + if CometConf.COMET_EXEC_UNION_ENABLED.get(conf) && + u.children.forall(isCometNative) => newPlanWithProto( u, { val cometOp = CometUnionExec(u, u.output, u.children) CometSinkPlaceHolder(_, u, cometOp) }) + case u: UnionExec if !CometConf.COMET_EXEC_UNION_ENABLED.get(conf) => + withInfo(u, "Union is not enabled") + + case op: UnionExec if !op.children.forall(isCometNative) => + op + // For AQE broadcast stage on a Comet broadcast exchange case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) @@ -395,11 +429,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { _) => newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast + // exchange. It is only used for Comet native execution. We only transform Spark broadcast + // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the + // broadcast exchange is forced to be enabled by Comet config. case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => - // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast - // exchange. It is only used for Comet native execution. We only transform Spark broadcast - // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the - // broadcast exchange is forced to be enabled by Comet config. val newChildren = plan.children.map { case b: BroadcastExchangeExec if isCometNative(b.child) && @@ -412,7 +446,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } case other => other } - val newPlan = if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { + if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { val newPlan = convertNode(plan.withNewChildren(newChildren)) if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { newPlan @@ -428,20 +462,15 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { plan } - // this case should be checked only after the previous case checking for a - // child BroadcastExchange has been applied, otherwise that transform - // never gets applied - if (newPlan.children.forall(isCometNative)) { - // it seems odd to check this last, but that is how the original code was - // implemented - if (CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf)) { - newPlan - } else { - withInfo(op, "BroadcastHashJoin is not enabled") - } - } else { - op - } + // this case should be checked only after the previous case checking for a + // child BroadcastExchange has been applied, otherwise that transform + // never gets applied + case op: BroadcastHashJoinExec if !op.children.forall(isCometNative) => + op + + case op: BroadcastHashJoinExec + if !CometConf.COMET_EXEC_BROADCAST_HASH_JOIN_ENABLED.get(conf) => + withInfo(op, "BroadcastHashJoin is not enabled") // For AQE shuffle stage on a Comet shuffle exchange case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => @@ -505,9 +534,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { s } - case op: LocalTableScanExec => - newPlanWithProto(op, CometScanWrapper(_, CometLocalTableScanExec(op, op.rows, op.output))) - case op => op match { case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt index b6c8291e3b..5552d071da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) +* CometColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * CometColumnarToRow (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * CometColumnarToRow (17) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * CometColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -52,216 +57,233 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(6) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#7] -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] ReadSchema: struct -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#11] -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] ReadSchema: struct -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] +(17) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight +(18) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#16] -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] +(21) Union -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight +(22) BroadcastExchange +Input [1]: [customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) CometProject +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customer_sk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_county#19] +Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) + +(27) CometProject +Input [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] +(28) CometColumnarToRow [codegen id : 7] +Input [1]: [ca_address_sk#18] -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] +(29) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] +(31) Project [codegen id : 9] +Output [1]: [c_current_cdemo_sk#2] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) +(33) CometFilter +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#20) -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(34) CometProject +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(35) CometColumnarToRow [codegen id : 8] +Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight +(36) BroadcastExchange +Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(38) Project [codegen id : 9] +Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(39) HashAggregate [codegen id : 9] +Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] + +(40) CometColumnarExchange +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(41) CometColumnarToRow [codegen id : 10] +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(42) HashAggregate [codegen id : 10] +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +(43) RowToColumnar +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +(44) CometTakeOrderedAndProject +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_education_status#31 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#32 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#29,cd_marital_status#30,cd_education_status#31,cnt1#36,cd_purchase_estimate#24,cnt2#37,cd_credit_rating#32,cnt3#38,cd_dep_count#26,cnt4#39,cd_dep_employed_count#27,cnt5#40,cd_dep_college_count#28,cnt6#41]), [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41], 100, 0, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] + +(45) CometColumnarToRow [codegen id : 11] +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) +(47) CometFilter +Input [3]: [d_date_sk#7, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 7)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +(48) CometProject +Input [3]: [d_date_sk#7, d_year#42, d_moy#43] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..fbfef118e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -1,56 +1,67 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt index 38f41d10a5..5decf7b86d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt @@ -1,52 +1,78 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt index b6c8291e3b..5552d071da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) +* CometColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * CometColumnarToRow (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * CometColumnarToRow (17) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * CometColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -52,216 +57,233 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(6) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#7] -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] ReadSchema: struct -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#11] -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] ReadSchema: struct -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] +(17) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight +(18) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#16] -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] +(21) Union -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight +(22) BroadcastExchange +Input [1]: [customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) CometProject +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customer_sk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_county#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_county#19] +Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) + +(27) CometProject +Input [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] +(28) CometColumnarToRow [codegen id : 7] +Input [1]: [ca_address_sk#18] -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] +(29) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] +(31) Project [codegen id : 9] +Output [1]: [c_current_cdemo_sk#2] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) +(33) CometFilter +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#20) -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(34) CometProject +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(35) CometColumnarToRow [codegen id : 8] +Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight +(36) BroadcastExchange +Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(38) Project [codegen id : 9] +Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(39) HashAggregate [codegen id : 9] +Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] + +(40) CometColumnarExchange +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(41) CometColumnarToRow [codegen id : 10] +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +(42) HashAggregate [codegen id : 10] +Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +(43) RowToColumnar +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +(44) CometTakeOrderedAndProject +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_education_status#31 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#32 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#29,cd_marital_status#30,cd_education_status#31,cnt1#36,cd_purchase_estimate#24,cnt2#37,cd_credit_rating#32,cnt3#38,cd_dep_count#26,cnt4#39,cd_dep_employed_count#27,cnt5#40,cd_dep_college_count#28,cnt6#41]), [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41], 100, 0, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] + +(45) CometColumnarToRow [codegen id : 11] +Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#42, d_moy#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) +(47) CometFilter +Input [3]: [d_date_sk#7, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 7)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +(48) CometProject +Input [3]: [d_date_sk#7, d_year#42, d_moy#43] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt index 2cdc75e15e..fbfef118e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt @@ -1,56 +1,67 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 38f41d10a5..5decf7b86d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -1,52 +1,78 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometProject [cs_ship_customer_sk] [customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt index 49c1c8e0d7..7a08937760 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt @@ -1,76 +1,85 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) +* CometColumnarToRow (81) ++- CometTakeOrderedAndProject (80) + +- RowToColumnar (79) + +- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * CometColumnarToRow (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (26) + : : : +- * CometColumnarToRow (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : +- ReusedExchange (29) + : +- BroadcastExchange (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometFilter (39) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (48) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * CometColumnarToRow (74) + +- CometColumnarExchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * CometColumnarToRow (62) + : : +- CometProject (61) + : : +- CometFilter (60) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometFilter (64) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (69) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -88,7 +97,10 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 3] +Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -96,375 +108,414 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#15) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] + +(8) BroadcastExchange Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#15] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -(14) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] +Aggregate Attributes [1]: [sum#22] +Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(15) CometColumnarExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] +(16) CometColumnarToRow [codegen id : 16] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] + +(17) HashAggregate [codegen id : 16] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] +Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) +(18) Filter [codegen id : 16] +Input [2]: [customer_id#25, year_total#26] +Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) +(20) CometFilter +Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) + +(21) CometProject +Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] +(22) CometColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +(24) CometFilter +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Condition : isnotnull(ss_customer_sk#35) + +(25) CometColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(26) BroadcastExchange +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#27] +Right keys [1]: [ss_customer_sk#35] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(29) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#40, d_year#41] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 6] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] +Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] + +(32) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] + +(33) CometColumnarExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometColumnarToRow [codegen id : 7] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] + +(35) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] +Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] + +(36) BroadcastExchange +Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#44] +Join type: Inner +Join condition: None + +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) +(39) CometFilter +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] +(40) CometProject +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +(41) CometColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight +(43) CometFilter +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Condition : isnotnull(ws_bill_customer_sk#63) -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] +(44) CometColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] +(45) BroadcastExchange +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#49] +Right keys [1]: [ws_bill_customer_sk#63] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 10] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] + +(48) ReusedExchange [Reuses operator id: 85] +Output [2]: [d_date_sk#68, d_year#69] + +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 10] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#68, d_year#69] + +(51) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] +Aggregate Attributes [1]: [sum#70] +Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] + +(52) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometColumnarToRow [codegen id : 11] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] + +(54) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72] +Results [2]: [c_customer_id#57 AS customer_id#73, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72,18,2) AS year_total#74] + +(55) Filter [codegen id : 11] +Input [2]: [customer_id#73, year_total#74] +Condition : (isnotnull(year_total#74) AND (year_total#74 > 0.00)) + +(56) BroadcastExchange +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#73] +Join type: Inner +Join condition: None + +(58) Project [codegen id : 16] +Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74] +Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#73, year_total#74] + +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) +(60) CometFilter +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Condition : (isnotnull(c_customer_sk#75) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true))) -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] +(61) CometProject +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Arguments: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62], [c_customer_sk#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#77, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#78, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#79, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#80, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#81, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#82, 50, true, false, true) AS c_email_address#62] + +(62) CometColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] +(64) CometFilter +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_bill_customer_sk#83) + +(65) CometColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] + +(66) BroadcastExchange +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#75] +Right keys [1]: [ws_bill_customer_sk#83] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [12]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] + +(69) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#88, d_year#89] + +(70) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(71) Project [codegen id : 14] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] + +(72) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] + +(73) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometColumnarToRow [codegen id : 15] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] + +(75) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72] +Results [2]: [c_customer_id#57 AS customer_id#92, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72,18,2) AS year_total#93] -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] +(76) BroadcastExchange +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#92] +Join type: Inner +Join condition: (CASE WHEN (year_total#74 > 0.00) THEN (year_total#93 / year_total#74) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight +(78) Project [codegen id : 16] +Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74, customer_id#92, year_total#93] -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(79) RowToColumnar +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(80) CometTakeOrderedAndProject +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#44 ASC NULLS FIRST,customer_first_name#45 ASC NULLS FIRST,customer_last_name#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#44,customer_first_name#45,customer_last_name#46,customer_email_address#47]), [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47], 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(81) CometColumnarToRow [codegen id : 17] +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometFilter (83) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(83) CometFilter Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(76) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +(87) CometFilter +Input [2]: [d_date_sk#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] +(88) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#40, d_year#41] -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(89) BroadcastExchange +Input [2]: [d_date_sk#40, d_year#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#39 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index 476c7be954..f5dda0de66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -1,90 +1,107 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt index 157d1d587e..0e65a6a74f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt @@ -1,90 +1,134 @@ -WholeStageCodegen (1) +WholeStageCodegen (17) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + RowToColumnar + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name,customer_email_address] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year] #4 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year] #9 + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt index 49c1c8e0d7..7a08937760 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt @@ -1,76 +1,85 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) +* CometColumnarToRow (81) ++- CometTakeOrderedAndProject (80) + +- RowToColumnar (79) + +- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * CometColumnarToRow (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (26) + : : : +- * CometColumnarToRow (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : +- ReusedExchange (29) + : +- BroadcastExchange (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometFilter (39) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (48) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * CometColumnarToRow (74) + +- CometColumnarExchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * CometColumnarToRow (62) + : : +- CometProject (61) + : : +- CometFilter (60) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometFilter (64) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (69) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -88,7 +97,10 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 3] +Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -96,375 +108,414 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#15) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] + +(8) BroadcastExchange Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#15] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -(14) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] +Aggregate Attributes [1]: [sum#22] +Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(15) CometColumnarExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] +(16) CometColumnarToRow [codegen id : 16] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] + +(17) HashAggregate [codegen id : 16] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] +Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) +(18) Filter [codegen id : 16] +Input [2]: [customer_id#25, year_total#26] +Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) +(20) CometFilter +Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) + +(21) CometProject +Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] +(22) CometColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +(24) CometFilter +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Condition : isnotnull(ss_customer_sk#35) + +(25) CometColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(26) BroadcastExchange +Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#27] +Right keys [1]: [ss_customer_sk#35] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(29) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#40, d_year#41] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 6] +Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] +Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] + +(32) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] + +(33) CometColumnarExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometColumnarToRow [codegen id : 7] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] + +(35) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] +Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] + +(36) BroadcastExchange +Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#44] +Join type: Inner +Join condition: None + +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) +(39) CometFilter +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] +(40) CometProject +Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +(41) CometColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight +(43) CometFilter +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Condition : isnotnull(ws_bill_customer_sk#63) -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] +(44) CometColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] +(45) BroadcastExchange +Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#49] +Right keys [1]: [ws_bill_customer_sk#63] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 10] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] + +(48) ReusedExchange [Reuses operator id: 85] +Output [2]: [d_date_sk#68, d_year#69] + +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#68] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 10] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#68, d_year#69] + +(51) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] +Aggregate Attributes [1]: [sum#70] +Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] + +(52) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometColumnarToRow [codegen id : 11] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] + +(54) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72] +Results [2]: [c_customer_id#57 AS customer_id#73, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72,18,2) AS year_total#74] + +(55) Filter [codegen id : 11] +Input [2]: [customer_id#73, year_total#74] +Condition : (isnotnull(year_total#74) AND (year_total#74 > 0.00)) + +(56) BroadcastExchange +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#73] +Join type: Inner +Join condition: None + +(58) Project [codegen id : 16] +Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74] +Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#73, year_total#74] + +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) +(60) CometFilter +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Condition : (isnotnull(c_customer_sk#75) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true))) -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] +(61) CometProject +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Arguments: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62], [c_customer_sk#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#77, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#78, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#79, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#80, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#81, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#82, 50, true, false, true) AS c_email_address#62] + +(62) CometColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] +(64) CometFilter +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_bill_customer_sk#83) + +(65) CometColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] + +(66) BroadcastExchange +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#75] +Right keys [1]: [ws_bill_customer_sk#83] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [12]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] + +(69) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#88, d_year#89] + +(70) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(71) Project [codegen id : 14] +Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] + +(72) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] + +(73) CometColumnarExchange +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] +Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometColumnarToRow [codegen id : 15] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] + +(75) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] +Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72] +Results [2]: [c_customer_id#57 AS customer_id#92, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72,18,2) AS year_total#93] -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] +(76) BroadcastExchange +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#25] +Right keys [1]: [customer_id#92] +Join type: Inner +Join condition: (CASE WHEN (year_total#74 > 0.00) THEN (year_total#93 / year_total#74) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight +(78) Project [codegen id : 16] +Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74, customer_id#92, year_total#93] -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(79) RowToColumnar +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(80) CometTakeOrderedAndProject +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#44 ASC NULLS FIRST,customer_first_name#45 ASC NULLS FIRST,customer_last_name#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#44,customer_first_name#45,customer_last_name#46,customer_email_address#47]), [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47], 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +(81) CometColumnarToRow [codegen id : 17] +Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometFilter (83) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(83) CometFilter Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(76) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +(87) CometFilter +Input [2]: [d_date_sk#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] +(88) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#40, d_year#41] -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(89) BroadcastExchange +Input [2]: [d_date_sk#40, d_year#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#39 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt index 476c7be954..f5dda0de66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt @@ -1,90 +1,107 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index 157d1d587e..0e65a6a74f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -1,90 +1,134 @@ -WholeStageCodegen (1) +WholeStageCodegen (17) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + RowToColumnar + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name,customer_email_address] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year] #4 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year] #9 + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt index 410aecff1b..58d7150a81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- RowToColumnar (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -36,129 +38,134 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(16) CometExchange +(15) CometColumnarExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometHashAggregate +(16) CometColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] + +(23) RowToColumnar +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(24) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(27) CometFilter +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(28) CometProject +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 9ddd3aeb38..9c95a71e3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -1,31 +1,37 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt index 58ac81ba0d..c4ddd74416 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt @@ -1,35 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + RowToColumnar + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_class] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt index 410aecff1b..58d7150a81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- RowToColumnar (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -36,129 +38,134 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(16) CometExchange +(15) CometColumnarExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometHashAggregate +(16) CometColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] + +(23) RowToColumnar +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(24) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(27) CometFilter +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(28) CometProject +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt index 9ddd3aeb38..9c95a71e3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt @@ -1,31 +1,37 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index 58ac81ba0d..c4ddd74416 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -1,35 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + RowToColumnar + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_class] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt index ea550c6c74..5f8639ff9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt @@ -1,89 +1,93 @@ == Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +* CometColumnarToRow (89) ++- CometTakeOrderedAndProject (88) + +- RowToColumnar (87) + +- * BroadcastHashJoin Inner BuildRight (86) + :- * Filter (68) + : +- * HashAggregate (67) + : +- * CometColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- * HashAggregate (64) + : +- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) + : :- * Project (60) + : : +- * BroadcastHashJoin Inner BuildRight (59) + : : :- * BroadcastHashJoin LeftSemi BuildRight (52) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * CometColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * CometColumnarToRow (36) + : : : : +- CometHashAggregate (35) + : : : : +- CometColumnarExchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * CometColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * CometColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * CometColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * CometColumnarToRow (39) + : : : : : +- CometFilter (38) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (58) + : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : :- * CometColumnarToRow (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) + : : +- ReusedExchange (56) + : +- ReusedExchange (61) + +- BroadcastExchange (85) + +- * Filter (84) + +- * HashAggregate (83) + +- * CometColumnarToRow (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- * Project (79) + +- * BroadcastHashJoin Inner BuildRight (78) + :- * Project (76) + : +- * BroadcastHashJoin Inner BuildRight (75) + : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : :- * CometColumnarToRow (71) + : : : +- CometFilter (70) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (69) + : : +- ReusedExchange (72) + : +- ReusedExchange (74) + +- ReusedExchange (77) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -98,18 +102,24 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(6) CometColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -117,22 +127,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(8) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(9) CometColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(8) CometFilter +(11) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(12) CometColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -140,604 +156,622 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(10) CometFilter +(14) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(15) CometColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(17) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(13) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 1] Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] +(22) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#24] -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None -(21) CometProject +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(22) CometBroadcastExchange +(25) BroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None -(24) CometBroadcastExchange +(27) BroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(26) CometProject +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] +(30) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#25] -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(34) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(35) CometHashAggregate +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +(36) CometColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] +(38) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(39) CometColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(40) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(43) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#36] + +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(46) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(47) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(48) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(51) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight +(52) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) +(54) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] +(55) CometColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight +(56) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#37] -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(57) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight +(58) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +(59) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] +(60) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] +(61) ReusedExchange [Reuses operator id: 114] +Output [1]: [d_date_sk#42] -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +(63) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(64) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(65) CometColumnarExchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(66) CometColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(68) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +(70) CometFilter +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) + +(71) CometColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] + +(72) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#61] + +(73) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(74) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(76) Project [codegen id : 50] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(77) ReusedExchange [Reuses operator id: 128] +Output [1]: [d_date_sk#66] + +(78) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 50] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(80) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(81) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(82) CometColumnarToRow [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(83) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(84) Filter [codegen id : 51] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(85) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] + +(86) BroadcastHashJoin [codegen id : 52] +Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Join type: Inner +Join condition: None + +(87) RowToColumnar +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +(88) CometTakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sales#52,number_sales#53,channel#75,i_brand_id#63,i_class_id#64,i_category_id#65,sales#76,number_sales#77]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77], 100, 0, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +(89) CometColumnarToRow [codegen id : 53] +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (109) ++- * CometColumnarToRow (108) + +- CometColumnarExchange (107) + +- * HashAggregate (106) + +- Union (105) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * CometColumnarToRow (91) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (90) + : +- ReusedExchange (92) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * CometColumnarToRow (96) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (97) + +- * Project (104) + +- * BroadcastHashJoin Inner BuildRight (103) + :- * CometColumnarToRow (101) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (100) + +- ReusedExchange (102) + + +(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] +(91) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] + +(92) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#82] -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight +(93) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#80] +Right keys [1]: [d_date_sk#82] +Join type: Inner +Join condition: None -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] +(94) Project [codegen id : 2] +Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] +(96) CometColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] + +(97) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#89] -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight +(98) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#87] +Right keys [1]: [d_date_sk#89] +Join type: Inner +Join condition: None -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] +(99) Project [codegen id : 4] +Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +(100) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] +(101) CometColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight +(102) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#96] -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] +(103) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#96] +Join type: Inner +Join condition: None -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] +(104) Project [codegen id : 6] +Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] +(105) Union + +(106) HashAggregate [codegen id : 7] +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [2]: [sum#99, count#100] +Results [2]: [sum#101, count#102] -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(107) CometColumnarExchange +Input [2]: [sum#101, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] +(108) CometColumnarToRow [codegen id : 8] +Input [2]: [sum#101, count#102] -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] +(109) HashAggregate [codegen id : 8] +Input [2]: [sum#101, count#102] +Keys: [] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) +BroadcastExchange (114) ++- * CometColumnarToRow (113) + +- CometProject (112) + +- CometFilter (111) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (110) -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] +(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#42, d_week_seq#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) +(111) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#105] +Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] +(112) CometProject +Input [2]: [d_date_sk#42, d_week_seq#105] +Arguments: [d_date_sk#42], [d_date_sk#42] -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] +(113) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(114) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) +Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#106, [id=#107] +* CometColumnarToRow (118) ++- CometProject (117) + +- CometFilter (116) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (115) -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) +(116) CometFilter +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] +(117) CometProject +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Arguments: [d_week_seq#108], [d_week_seq#108] -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] +(118) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#108] -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) +Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (123) ++- * CometColumnarToRow (122) + +- CometProject (121) + +- CometFilter (120) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (119) -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] +(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] +(120) CometFilter +Input [2]: [d_date_sk#25, d_year#112] +Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] +(121) CometProject +Input [2]: [d_date_sk#25, d_year#112] +Arguments: [d_date_sk#25], [d_date_sk#25] -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(122) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +(123) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) +Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometProject (126) + +- CometFilter (125) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (124) -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] +(124) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) +(125) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#113] +Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] +(126) CometProject +Input [2]: [d_date_sk#66, d_week_seq#113] +Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(127) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(128) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) +Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#114, [id=#115] +* CometColumnarToRow (132) ++- CometProject (131) + +- CometFilter (130) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] +(130) CometFilter +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] +(131) CometProject +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Arguments: [d_week_seq#116], [d_week_seq#116] -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +(132) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#116] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index f56d229b68..88a9611f1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -1,339 +1,406 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 160 out of 333 eligible operators (48%). Final plan contains 71 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt index fb9abae378..b038a65567 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt @@ -1,153 +1,210 @@ -WholeStageCodegen (1) +WholeStageCodegen (53) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (52) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - CometHashAggregate [brand_id,class_id,category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (51) + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt index ea550c6c74..5f8639ff9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt @@ -1,89 +1,93 @@ == Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +* CometColumnarToRow (89) ++- CometTakeOrderedAndProject (88) + +- RowToColumnar (87) + +- * BroadcastHashJoin Inner BuildRight (86) + :- * Filter (68) + : +- * HashAggregate (67) + : +- * CometColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- * HashAggregate (64) + : +- * Project (63) + : +- * BroadcastHashJoin Inner BuildRight (62) + : :- * Project (60) + : : +- * BroadcastHashJoin Inner BuildRight (59) + : : :- * BroadcastHashJoin LeftSemi BuildRight (52) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (51) + : : : +- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * CometColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (48) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : :- * CometColumnarToRow (36) + : : : : +- CometHashAggregate (35) + : : : : +- CometColumnarExchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * CometColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * CometColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * CometColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * CometColumnarToRow (39) + : : : : : +- CometFilter (38) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) + : : : : +- ReusedExchange (40) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (58) + : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : :- * CometColumnarToRow (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) + : : +- ReusedExchange (56) + : +- ReusedExchange (61) + +- BroadcastExchange (85) + +- * Filter (84) + +- * HashAggregate (83) + +- * CometColumnarToRow (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- * Project (79) + +- * BroadcastHashJoin Inner BuildRight (78) + :- * Project (76) + : +- * BroadcastHashJoin Inner BuildRight (75) + : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : :- * CometColumnarToRow (71) + : : : +- CometFilter (70) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (69) + : : +- ReusedExchange (72) + : +- ReusedExchange (74) + +- ReusedExchange (77) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -98,18 +102,24 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(6) CometColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -117,22 +127,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(8) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(9) CometColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(8) CometFilter +(11) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(12) CometColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -140,604 +156,622 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(10) CometFilter +(14) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(15) CometColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(17) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(13) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 1] Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] +(22) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#24] -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None -(21) CometProject +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(22) CometBroadcastExchange +(25) BroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None -(24) CometBroadcastExchange +(27) BroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(26) CometProject +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] +(30) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#25] -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(34) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(35) CometHashAggregate +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +(36) CometColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] +(38) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(39) CometColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(40) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(43) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#36] + +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(46) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(47) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(48) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(51) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight +(52) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) +(54) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] +(55) CometColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight +(56) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#37] -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(57) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight +(58) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +(59) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] +(60) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] +(61) ReusedExchange [Reuses operator id: 114] +Output [1]: [d_date_sk#42] -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +(63) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(64) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(65) CometColumnarExchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(66) CometColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(67) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(68) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +(70) CometFilter +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) + +(71) CometColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] + +(72) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#61] + +(73) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(74) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(76) Project [codegen id : 50] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(77) ReusedExchange [Reuses operator id: 128] +Output [1]: [d_date_sk#66] + +(78) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 50] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(80) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(81) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(82) CometColumnarToRow [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(83) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(84) Filter [codegen id : 51] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(85) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] + +(86) BroadcastHashJoin [codegen id : 52] +Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Join type: Inner +Join condition: None + +(87) RowToColumnar +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +(88) CometTakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sales#52,number_sales#53,channel#75,i_brand_id#63,i_class_id#64,i_category_id#65,sales#76,number_sales#77]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77], 100, 0, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +(89) CometColumnarToRow [codegen id : 53] +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (109) ++- * CometColumnarToRow (108) + +- CometColumnarExchange (107) + +- * HashAggregate (106) + +- Union (105) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * CometColumnarToRow (91) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (90) + : +- ReusedExchange (92) + :- * Project (99) + : +- * BroadcastHashJoin Inner BuildRight (98) + : :- * CometColumnarToRow (96) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (95) + : +- ReusedExchange (97) + +- * Project (104) + +- * BroadcastHashJoin Inner BuildRight (103) + :- * CometColumnarToRow (101) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (100) + +- ReusedExchange (102) + + +(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] +(91) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] + +(92) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#82] -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight +(93) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#80] +Right keys [1]: [d_date_sk#82] +Join type: Inner +Join condition: None -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] +(94) Project [codegen id : 2] +Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] +(96) CometColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] + +(97) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#89] -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight +(98) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#87] +Right keys [1]: [d_date_sk#89] +Join type: Inner +Join condition: None -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] +(99) Project [codegen id : 4] +Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +(100) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] +(101) CometColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight +(102) ReusedExchange [Reuses operator id: 123] +Output [1]: [d_date_sk#96] -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] +(103) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#96] +Join type: Inner +Join condition: None -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] +(104) Project [codegen id : 6] +Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] +(105) Union + +(106) HashAggregate [codegen id : 7] +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [2]: [sum#99, count#100] +Results [2]: [sum#101, count#102] -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(107) CometColumnarExchange +Input [2]: [sum#101, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] +(108) CometColumnarToRow [codegen id : 8] +Input [2]: [sum#101, count#102] -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] +(109) HashAggregate [codegen id : 8] +Input [2]: [sum#101, count#102] +Keys: [] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) +BroadcastExchange (114) ++- * CometColumnarToRow (113) + +- CometProject (112) + +- CometFilter (111) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (110) -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] +(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#42, d_week_seq#105] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) +(111) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#105] +Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] +(112) CometProject +Input [2]: [d_date_sk#42, d_week_seq#105] +Arguments: [d_date_sk#42], [d_date_sk#42] -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] +(113) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(114) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) +Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#106, [id=#107] +* CometColumnarToRow (118) ++- CometProject (117) + +- CometFilter (116) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (115) -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) +(116) CometFilter +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] +(117) CometProject +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Arguments: [d_week_seq#108], [d_week_seq#108] -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] +(118) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#108] -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) +Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (123) ++- * CometColumnarToRow (122) + +- CometProject (121) + +- CometFilter (120) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (119) -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] +(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] +(120) CometFilter +Input [2]: [d_date_sk#25, d_year#112] +Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] +(121) CometProject +Input [2]: [d_date_sk#25, d_year#112] +Arguments: [d_date_sk#25], [d_date_sk#25] -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(122) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +(123) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) +Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometProject (126) + +- CometFilter (125) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (124) -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] +(124) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) +(125) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#113] +Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] +(126) CometProject +Input [2]: [d_date_sk#66, d_week_seq#113] +Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(127) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(128) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) +Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#114, [id=#115] +* CometColumnarToRow (132) ++- CometProject (131) + +- CometFilter (130) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] +(130) CometFilter +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] +(131) CometProject +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Arguments: [d_week_seq#116], [d_week_seq#116] -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +(132) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#116] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt index f56d229b68..88a9611f1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt @@ -1,339 +1,406 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 160 out of 333 eligible operators (48%). Final plan contains 71 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index fb9abae378..b038a65567 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -1,153 +1,210 @@ -WholeStageCodegen (1) +WholeStageCodegen (53) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk] #10 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (52) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #4 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - CometHashAggregate [brand_id,class_id,category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (51) + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt index a727c5e743..7d029ef698 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt @@ -1,126 +1,143 @@ == Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) +* CometColumnarToRow (139) ++- CometTakeOrderedAndProject (138) + +- CometHashAggregate (137) + +- CometColumnarExchange (136) + +- * HashAggregate (135) + +- Union (134) + :- * HashAggregate (105) + : +- * CometColumnarToRow (104) + : +- CometColumnarExchange (103) + : +- * HashAggregate (102) + : +- Union (101) + : :- * Filter (68) + : : +- * HashAggregate (67) + : : +- * CometColumnarToRow (66) + : : +- CometColumnarExchange (65) + : : +- * HashAggregate (64) + : : +- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (51) + : : : : +- * Project (50) + : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : :- * CometColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (48) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : : :- * CometColumnarToRow (36) + : : : : : +- CometHashAggregate (35) + : : : : : +- CometColumnarExchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * CometColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * CometColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * CometColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (46) + : : : : +- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Project (42) + : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : :- * CometColumnarToRow (39) + : : : : : : +- CometFilter (38) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) + : : : : : +- ReusedExchange (40) + : : : : +- ReusedExchange (43) + : : : +- BroadcastExchange (58) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : :- * CometColumnarToRow (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) + : : : +- ReusedExchange (56) + : : +- ReusedExchange (61) + : :- * Filter (84) + : : +- * HashAggregate (83) + : : +- * CometColumnarToRow (82) + : : +- CometColumnarExchange (81) + : : +- * HashAggregate (80) + : : +- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * Project (76) + : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : : : :- * CometColumnarToRow (71) + : : : : : +- CometFilter (70) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) + : : : : +- ReusedExchange (72) + : : : +- ReusedExchange (74) + : : +- ReusedExchange (77) + : +- * Filter (100) + : +- * HashAggregate (99) + : +- * CometColumnarToRow (98) + : +- CometColumnarExchange (97) + : +- * HashAggregate (96) + : +- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * BroadcastHashJoin LeftSemi BuildRight (89) + : : : :- * CometColumnarToRow (87) + : : : : +- CometFilter (86) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (85) + : : : +- ReusedExchange (88) + : : +- ReusedExchange (90) + : +- ReusedExchange (93) + :- * HashAggregate (112) + : +- * CometColumnarToRow (111) + : +- CometColumnarExchange (110) + : +- * HashAggregate (109) + : +- * HashAggregate (108) + : +- * CometColumnarToRow (107) + : +- ReusedExchange (106) + :- * HashAggregate (119) + : +- * CometColumnarToRow (118) + : +- CometColumnarExchange (117) + : +- * HashAggregate (116) + : +- * HashAggregate (115) + : +- * CometColumnarToRow (114) + : +- ReusedExchange (113) + :- * HashAggregate (126) + : +- * CometColumnarToRow (125) + : +- CometColumnarExchange (124) + : +- * HashAggregate (123) + : +- * HashAggregate (122) + : +- * CometColumnarToRow (121) + : +- ReusedExchange (120) + +- * HashAggregate (133) + +- * CometColumnarToRow (132) + +- CometColumnarExchange (131) + +- * HashAggregate (130) + +- * HashAggregate (129) + +- * CometColumnarToRow (128) + +- ReusedExchange (127) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -135,18 +152,24 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(6) CometColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -154,22 +177,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(8) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(9) CometColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(8) CometFilter +(11) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(12) CometColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -177,738 +206,815 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(10) CometFilter +(14) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(15) CometColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(17) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(13) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 1] Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] +(22) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#24] -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None -(21) CometProject +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(22) CometBroadcastExchange +(25) BroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None -(24) CometBroadcastExchange +(27) BroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(26) CometProject +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] +(30) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#25] -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(34) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(35) CometHashAggregate +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +(36) CometColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(38) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(39) CometColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(40) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(43) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#36] + +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(46) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(47) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(48) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(51) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(52) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) +(54) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : isnotnull(i_item_sk#38) -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] +(55) CometColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight +(56) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#37] -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(57) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight +(58) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +(59) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) +(60) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] +(61) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#42] -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] +(62) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight +(63) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(64) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(65) CometColumnarExchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(66) CometColumnarToRow [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(67) HashAggregate [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(68) Filter [codegen id : 26] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] +PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +(70) CometFilter +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Condition : isnotnull(cs_item_sk#56) + +(71) CometColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] + +(72) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#61] + +(73) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(74) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(76) Project [codegen id : 51] +Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(77) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#66] + +(78) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 51] +Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(80) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(81) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(82) CometColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(83) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] +Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(84) Filter [codegen id : 52] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#110, number_sales#111] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] - -(110) CometExchange -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#118, number_sales#119] +(86) CometFilter +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_item_sk#78) + +(87) CometColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] + +(88) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#83] + +(89) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [ss_item_sk#83] +Join type: LeftSemi +Join condition: None + +(90) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(91) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [i_item_sk#84] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 77] +Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(93) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#88] + +(94) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 77] +Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] + +(96) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] + +(97) CometColumnarExchange +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(98) CometColumnarToRow [codegen id : 78] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] + +(99) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] +Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] + +(100) Filter [codegen id : 78] +Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] +Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(101) Union + +(102) HashAggregate [codegen id : 79] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] +Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] + +(103) CometColumnarExchange +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(104) CometColumnarToRow [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] + +(105) HashAggregate [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] + +(106) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] + +(107) CometColumnarToRow [codegen id : 160] +Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] + +(108) HashAggregate [codegen id : 160] +Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [5]: [channel#51, i_brand_id#110, i_class_id#111, sum(sales#52)#106 AS sum_sales#113, sum(number_sales#53)#107 AS number_sales#114] + +(109) HashAggregate [codegen id : 160] +Input [5]: [channel#51, i_brand_id#110, i_class_id#111, sum_sales#113, number_sales#114] +Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] +Functions [2]: [partial_sum(sum_sales#113), partial_sum(number_sales#114)] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] + +(110) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#51, i_brand_id#110, i_class_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(111) CometColumnarToRow [codegen id : 161] +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] + +(112) HashAggregate [codegen id : 161] +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] +Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] +Functions [2]: [sum(sum_sales#113), sum(number_sales#114)] +Aggregate Attributes [2]: [sum(sum_sales#113)#121, sum(number_sales#114)#122] +Results [6]: [channel#51, i_brand_id#110, i_class_id#111, null AS i_category_id#123, sum(sum_sales#113)#121 AS sum(sum_sales)#124, sum(number_sales#114)#122 AS sum(number_sales)#125] + +(113) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] + +(114) CometColumnarToRow [codegen id : 241] +Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] + +(115) HashAggregate [codegen id : 241] +Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [4]: [channel#51, i_brand_id#126, sum(sales#52)#106 AS sum_sales#129, sum(number_sales#53)#107 AS number_sales#130] + +(116) HashAggregate [codegen id : 241] +Input [4]: [channel#51, i_brand_id#126, sum_sales#129, number_sales#130] +Keys [2]: [channel#51, i_brand_id#126] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] +Results [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] + +(117) CometColumnarExchange +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] +Arguments: hashpartitioning(channel#51, i_brand_id#126, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(118) CometColumnarToRow [codegen id : 242] +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] + +(119) HashAggregate [codegen id : 242] +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] +Keys [2]: [channel#51, i_brand_id#126] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#137, sum(number_sales#130)#138] +Results [6]: [channel#51, i_brand_id#126, null AS i_class_id#139, null AS i_category_id#140, sum(sum_sales#129)#137 AS sum(sum_sales)#141, sum(number_sales#130)#138 AS sum(number_sales)#142] + +(120) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] + +(121) CometColumnarToRow [codegen id : 322] +Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] + +(122) HashAggregate [codegen id : 322] +Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [3]: [channel#51, sum(sales#52)#106 AS sum_sales#146, sum(number_sales#53)#107 AS number_sales#147] + +(123) HashAggregate [codegen id : 322] +Input [3]: [channel#51, sum_sales#146, number_sales#147] +Keys [1]: [channel#51] +Functions [2]: [partial_sum(sum_sales#146), partial_sum(number_sales#147)] +Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] +Results [4]: [channel#51, sum#151, isEmpty#152, sum#153] + +(124) CometColumnarExchange +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(125) CometColumnarToRow [codegen id : 323] +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] + +(126) HashAggregate [codegen id : 323] +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] +Keys [1]: [channel#51] +Functions [2]: [sum(sum_sales#146), sum(number_sales#147)] +Aggregate Attributes [2]: [sum(sum_sales#146)#154, sum(number_sales#147)#155] +Results [6]: [channel#51, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#146)#154 AS sum(sum_sales)#159, sum(number_sales#147)#155 AS sum(number_sales)#160] + +(127) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] + +(128) CometColumnarToRow [codegen id : 403] +Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] + +(129) HashAggregate [codegen id : 403] +Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [2]: [sum(sales#52)#106 AS sum_sales#164, sum(number_sales#53)#107 AS number_sales#165] + +(130) HashAggregate [codegen id : 403] +Input [2]: [sum_sales#164, number_sales#165] Keys: [] -Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] +Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [3]: [sum#169, isEmpty#170, sum#171] + +(131) CometColumnarExchange +Input [3]: [sum#169, isEmpty#170, sum#171] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometExchange -Input [3]: [sum#120, isEmpty#121, sum#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(132) CometColumnarToRow [codegen id : 404] +Input [3]: [sum#169, isEmpty#170, sum#171] -(116) CometHashAggregate -Input [3]: [sum#120, isEmpty#121, sum#122] +(133) HashAggregate [codegen id : 404] +Input [3]: [sum#169, isEmpty#170, sum#171] Keys: [] -Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] -Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] -Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] -Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] +Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] +Results [6]: [null AS channel#174, null AS i_brand_id#175, null AS i_class_id#176, null AS i_category_id#177, sum(sum_sales#164)#172 AS sum(sum_sales)#178, sum(number_sales#165)#173 AS sum(number_sales)#179] + +(134) Union + +(135) HashAggregate [codegen id : 405] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(136) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(137) CometHashAggregate +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] Functions: [] -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(138) CometTakeOrderedAndProject +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#108,number_sales#109]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109], 100, 0, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(139) CometColumnarToRow [codegen id : 406] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] ===== Subqueries ===== -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (159) ++- * CometColumnarToRow (158) + +- CometColumnarExchange (157) + +- * HashAggregate (156) + +- Union (155) + :- * Project (144) + : +- * BroadcastHashJoin Inner BuildRight (143) + : :- * CometColumnarToRow (141) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (140) + : +- ReusedExchange (142) + :- * Project (149) + : +- * BroadcastHashJoin Inner BuildRight (148) + : :- * CometColumnarToRow (146) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (145) + : +- ReusedExchange (147) + +- * Project (154) + +- * BroadcastHashJoin Inner BuildRight (153) + :- * CometColumnarToRow (151) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (150) + +- ReusedExchange (152) + + +(140) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] +PartitionFilters: [isnotnull(ss_sold_date_sk#182), dynamicpruningexpression(ss_sold_date_sk#182 IN dynamicpruning#183)] ReadSchema: struct -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#147] +(141) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Right output [1]: [d_date_sk#147] -Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight +(142) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#184] -(126) CometProject -Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] -Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] +(143) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#182] +Right keys [1]: [d_date_sk#184] +Join type: Inner +Join condition: None -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] +(144) Project [codegen id : 2] +Output [2]: [ss_quantity#180 AS quantity#185, ss_list_price#181 AS list_price#186] +Input [4]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182, d_date_sk#184] + +(145) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] +PartitionFilters: [isnotnull(cs_sold_date_sk#189), dynamicpruningexpression(cs_sold_date_sk#189 IN dynamicpruning#190)] ReadSchema: struct -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) +(146) CometColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] -(130) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] +(147) ReusedExchange [Reuses operator id: 164] +Output [1]: [d_date_sk#191] -(131) CometBroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: [d_date_sk#154] +(148) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#189] +Right keys [1]: [d_date_sk#191] +Join type: Inner +Join condition: None -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Right output [1]: [d_date_sk#154] -Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight +(149) Project [codegen id : 4] +Output [2]: [cs_quantity#187 AS quantity#192, cs_list_price#188 AS list_price#193] +Input [4]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189, d_date_sk#191] -(133) CometProject -Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] -Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] +(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] +PartitionFilters: [isnotnull(ws_sold_date_sk#196), dynamicpruningexpression(ws_sold_date_sk#196 IN dynamicpruning#197)] ReadSchema: struct -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#162] +(151) CometColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] + +(152) ReusedExchange [Reuses operator id: 164] +Output [1]: [d_date_sk#198] -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Right output [1]: [d_date_sk#162] -Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight +(153) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#196] +Right keys [1]: [d_date_sk#198] +Join type: Inner +Join condition: None -(137) CometProject -Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] -Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] +(154) Project [codegen id : 6] +Output [2]: [ws_quantity#194 AS quantity#199, ws_list_price#195 AS list_price#200] +Input [4]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196, d_date_sk#198] -(138) CometUnion -Child 0 Input [2]: [quantity#148, list_price#149] -Child 1 Input [2]: [quantity#156, list_price#157] -Child 2 Input [2]: [quantity#163, list_price#164] +(155) Union -(139) CometHashAggregate -Input [2]: [quantity#148, list_price#149] +(156) HashAggregate [codegen id : 7] +Input [2]: [quantity#185, list_price#186] Keys: [] -Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] +Functions [1]: [partial_avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] +Aggregate Attributes [2]: [sum#201, count#202] +Results [2]: [sum#203, count#204] -(140) CometExchange -Input [2]: [sum#165, count#166] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(157) CometColumnarExchange +Input [2]: [sum#203, count#204] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(141) CometHashAggregate -Input [2]: [sum#165, count#166] -Keys: [] -Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] +(158) CometColumnarToRow [codegen id : 8] +Input [2]: [sum#203, count#204] -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#167] +(159) HashAggregate [codegen id : 8] +Input [2]: [sum#203, count#204] +Keys: [] +Functions [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] +Aggregate Attributes [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205] +Results [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205 AS average_sales#206] -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#182 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) +Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#189 IN dynamicpruning#190 +BroadcastExchange (164) ++- * CometColumnarToRow (163) + +- CometProject (162) + +- CometFilter (161) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (160) -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] +(160) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#191, d_year#207] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(144) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) +(161) CometFilter +Input [2]: [d_date_sk#191, d_year#207] +Condition : (((isnotnull(d_year#207) AND (d_year#207 >= 1998)) AND (d_year#207 <= 2000)) AND isnotnull(d_date_sk#191)) -(145) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] +(162) CometProject +Input [2]: [d_date_sk#191, d_year#207] +Arguments: [d_date_sk#191], [d_date_sk#191] -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#154] +(163) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#191] -(147) BroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(164) BroadcastExchange +Input [1]: [d_date_sk#191] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 +Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#196 IN dynamicpruning#190 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) +BroadcastExchange (169) ++- * CometColumnarToRow (168) + +- CometProject (167) + +- CometFilter (166) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (165) -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +(165) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#42, d_year#208, d_moy#209] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) +(166) CometFilter +Input [3]: [d_date_sk#42, d_year#208, d_moy#209] +Condition : ((((isnotnull(d_year#208) AND isnotnull(d_moy#209)) AND (d_year#208 = 2000)) AND (d_moy#209 = 11)) AND isnotnull(d_date_sk#42)) -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] +(167) CometProject +Input [3]: [d_date_sk#42, d_year#208, d_moy#209] +Arguments: [d_date_sk#42], [d_date_sk#42] -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] +(168) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(169) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (174) ++- * CometColumnarToRow (173) + +- CometProject (172) + +- CometFilter (171) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (170) -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#168] +(170) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#210] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#168] -Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) +(171) CometFilter +Input [2]: [d_date_sk#25, d_year#210] +Condition : (((isnotnull(d_year#210) AND (d_year#210 >= 1999)) AND (d_year#210 <= 2001)) AND isnotnull(d_date_sk#25)) -(155) CometProject -Input [2]: [d_date_sk#26, d_year#168] -Arguments: [d_date_sk#26], [d_date_sk#26] +(172) CometProject +Input [2]: [d_date_sk#25, d_year#210] +Arguments: [d_date_sk#25], [d_date_sk#25] -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] +(173) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(174) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index 83deb45432..13f50df6c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -1,2350 +1,2779 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 1067 out of 2302 eligible operators (46%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt index 758e917335..b3f0140800 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt @@ -1,179 +1,276 @@ -WholeStageCodegen (1) +WholeStageCodegen (406) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk] #12 - CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - CometHashAggregate [brand_id,class_id,category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (405) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (26) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (78) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (161) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (160) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (242) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id] #19 + WholeStageCodegen (241) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (323) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #20 + WholeStageCodegen (322) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (404) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange #21 + WholeStageCodegen (403) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt index a727c5e743..7d029ef698 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt @@ -1,126 +1,143 @@ == Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) +* CometColumnarToRow (139) ++- CometTakeOrderedAndProject (138) + +- CometHashAggregate (137) + +- CometColumnarExchange (136) + +- * HashAggregate (135) + +- Union (134) + :- * HashAggregate (105) + : +- * CometColumnarToRow (104) + : +- CometColumnarExchange (103) + : +- * HashAggregate (102) + : +- Union (101) + : :- * Filter (68) + : : +- * HashAggregate (67) + : : +- * CometColumnarToRow (66) + : : +- CometColumnarExchange (65) + : : +- * HashAggregate (64) + : : +- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (51) + : : : : +- * Project (50) + : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : :- * CometColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (48) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) + : : : : :- * CometColumnarToRow (36) + : : : : : +- CometHashAggregate (35) + : : : : : +- CometColumnarExchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * CometColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * CometColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * CometColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (46) + : : : : +- * Project (45) + : : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : : :- * Project (42) + : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : :- * CometColumnarToRow (39) + : : : : : : +- CometFilter (38) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) + : : : : : +- ReusedExchange (40) + : : : : +- ReusedExchange (43) + : : : +- BroadcastExchange (58) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) + : : : :- * CometColumnarToRow (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) + : : : +- ReusedExchange (56) + : : +- ReusedExchange (61) + : :- * Filter (84) + : : +- * HashAggregate (83) + : : +- * CometColumnarToRow (82) + : : +- CometColumnarExchange (81) + : : +- * HashAggregate (80) + : : +- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * Project (76) + : : : +- * BroadcastHashJoin Inner BuildRight (75) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) + : : : : :- * CometColumnarToRow (71) + : : : : : +- CometFilter (70) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) + : : : : +- ReusedExchange (72) + : : : +- ReusedExchange (74) + : : +- ReusedExchange (77) + : +- * Filter (100) + : +- * HashAggregate (99) + : +- * CometColumnarToRow (98) + : +- CometColumnarExchange (97) + : +- * HashAggregate (96) + : +- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * BroadcastHashJoin LeftSemi BuildRight (89) + : : : :- * CometColumnarToRow (87) + : : : : +- CometFilter (86) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (85) + : : : +- ReusedExchange (88) + : : +- ReusedExchange (90) + : +- ReusedExchange (93) + :- * HashAggregate (112) + : +- * CometColumnarToRow (111) + : +- CometColumnarExchange (110) + : +- * HashAggregate (109) + : +- * HashAggregate (108) + : +- * CometColumnarToRow (107) + : +- ReusedExchange (106) + :- * HashAggregate (119) + : +- * CometColumnarToRow (118) + : +- CometColumnarExchange (117) + : +- * HashAggregate (116) + : +- * HashAggregate (115) + : +- * CometColumnarToRow (114) + : +- ReusedExchange (113) + :- * HashAggregate (126) + : +- * CometColumnarToRow (125) + : +- CometColumnarExchange (124) + : +- * HashAggregate (123) + : +- * HashAggregate (122) + : +- * CometColumnarToRow (121) + : +- ReusedExchange (120) + +- * HashAggregate (133) + +- * CometColumnarToRow (132) + +- CometColumnarExchange (131) + +- * HashAggregate (130) + +- * HashAggregate (129) + +- * CometColumnarToRow (128) + +- ReusedExchange (127) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -135,18 +152,24 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(6) CometColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -154,22 +177,28 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(8) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(9) CometColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(8) CometFilter +(11) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(12) CometColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -177,738 +206,815 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(10) CometFilter +(14) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(15) CometColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(17) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(13) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 1] Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] +(22) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#24] -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None -(21) CometProject +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(22) CometBroadcastExchange +(25) BroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None -(24) CometBroadcastExchange +(27) BroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None -(26) CometProject +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] +(30) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#25] -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(34) CometColumnarExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] +(35) CometHashAggregate +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] Functions: [] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +(36) CometColumnarToRow [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(38) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(39) CometColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(40) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(43) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#36] + +(44) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(46) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(47) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(48) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(51) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(52) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) +(54) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : isnotnull(i_item_sk#38) -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] +(55) CometColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight +(56) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#37] -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +(57) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight +(58) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +(59) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) +(60) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] +(61) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#42] -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] +(62) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight +(63) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(64) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(65) CometColumnarExchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(66) CometColumnarToRow [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +(67) HashAggregate [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) +(68) Filter [codegen id : 26] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] +PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +(70) CometFilter +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Condition : isnotnull(cs_item_sk#56) + +(71) CometColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] + +(72) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#61] + +(73) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(74) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(76) Project [codegen id : 51] +Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(77) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#66] + +(78) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 51] +Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(80) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(81) CometColumnarExchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(82) CometColumnarToRow [codegen id : 52] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(83) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] +Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(84) Filter [codegen id : 52] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#110, number_sales#111] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] - -(110) CometExchange -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#118, number_sales#119] +(86) CometFilter +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_item_sk#78) + +(87) CometColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] + +(88) ReusedExchange [Reuses operator id: 51] +Output [1]: [ss_item_sk#83] + +(89) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [ss_item_sk#83] +Join type: LeftSemi +Join condition: None + +(90) ReusedExchange [Reuses operator id: 58] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(91) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [i_item_sk#84] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 77] +Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(93) ReusedExchange [Reuses operator id: 169] +Output [1]: [d_date_sk#88] + +(94) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 77] +Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] + +(96) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] + +(97) CometColumnarExchange +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(98) CometColumnarToRow [codegen id : 78] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] + +(99) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] +Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] + +(100) Filter [codegen id : 78] +Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] +Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(101) Union + +(102) HashAggregate [codegen id : 79] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] +Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] + +(103) CometColumnarExchange +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(104) CometColumnarToRow [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] + +(105) HashAggregate [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] + +(106) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] + +(107) CometColumnarToRow [codegen id : 160] +Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] + +(108) HashAggregate [codegen id : 160] +Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [5]: [channel#51, i_brand_id#110, i_class_id#111, sum(sales#52)#106 AS sum_sales#113, sum(number_sales#53)#107 AS number_sales#114] + +(109) HashAggregate [codegen id : 160] +Input [5]: [channel#51, i_brand_id#110, i_class_id#111, sum_sales#113, number_sales#114] +Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] +Functions [2]: [partial_sum(sum_sales#113), partial_sum(number_sales#114)] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] + +(110) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#51, i_brand_id#110, i_class_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(111) CometColumnarToRow [codegen id : 161] +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] + +(112) HashAggregate [codegen id : 161] +Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] +Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] +Functions [2]: [sum(sum_sales#113), sum(number_sales#114)] +Aggregate Attributes [2]: [sum(sum_sales#113)#121, sum(number_sales#114)#122] +Results [6]: [channel#51, i_brand_id#110, i_class_id#111, null AS i_category_id#123, sum(sum_sales#113)#121 AS sum(sum_sales)#124, sum(number_sales#114)#122 AS sum(number_sales)#125] + +(113) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] + +(114) CometColumnarToRow [codegen id : 241] +Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] + +(115) HashAggregate [codegen id : 241] +Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [4]: [channel#51, i_brand_id#126, sum(sales#52)#106 AS sum_sales#129, sum(number_sales#53)#107 AS number_sales#130] + +(116) HashAggregate [codegen id : 241] +Input [4]: [channel#51, i_brand_id#126, sum_sales#129, number_sales#130] +Keys [2]: [channel#51, i_brand_id#126] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] +Results [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] + +(117) CometColumnarExchange +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] +Arguments: hashpartitioning(channel#51, i_brand_id#126, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(118) CometColumnarToRow [codegen id : 242] +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] + +(119) HashAggregate [codegen id : 242] +Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] +Keys [2]: [channel#51, i_brand_id#126] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#137, sum(number_sales#130)#138] +Results [6]: [channel#51, i_brand_id#126, null AS i_class_id#139, null AS i_category_id#140, sum(sum_sales#129)#137 AS sum(sum_sales)#141, sum(number_sales#130)#138 AS sum(number_sales)#142] + +(120) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] + +(121) CometColumnarToRow [codegen id : 322] +Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] + +(122) HashAggregate [codegen id : 322] +Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [3]: [channel#51, sum(sales#52)#106 AS sum_sales#146, sum(number_sales#53)#107 AS number_sales#147] + +(123) HashAggregate [codegen id : 322] +Input [3]: [channel#51, sum_sales#146, number_sales#147] +Keys [1]: [channel#51] +Functions [2]: [partial_sum(sum_sales#146), partial_sum(number_sales#147)] +Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] +Results [4]: [channel#51, sum#151, isEmpty#152, sum#153] + +(124) CometColumnarExchange +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(125) CometColumnarToRow [codegen id : 323] +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] + +(126) HashAggregate [codegen id : 323] +Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] +Keys [1]: [channel#51] +Functions [2]: [sum(sum_sales#146), sum(number_sales#147)] +Aggregate Attributes [2]: [sum(sum_sales#146)#154, sum(number_sales#147)#155] +Results [6]: [channel#51, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#146)#154 AS sum(sum_sales)#159, sum(number_sales#147)#155 AS sum(number_sales)#160] + +(127) ReusedExchange [Reuses operator id: 103] +Output [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] + +(128) CometColumnarToRow [codegen id : 403] +Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] + +(129) HashAggregate [codegen id : 403] +Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [2]: [sum(sales#52)#106 AS sum_sales#164, sum(number_sales#53)#107 AS number_sales#165] + +(130) HashAggregate [codegen id : 403] +Input [2]: [sum_sales#164, number_sales#165] Keys: [] -Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] +Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [3]: [sum#169, isEmpty#170, sum#171] + +(131) CometColumnarExchange +Input [3]: [sum#169, isEmpty#170, sum#171] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometExchange -Input [3]: [sum#120, isEmpty#121, sum#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(132) CometColumnarToRow [codegen id : 404] +Input [3]: [sum#169, isEmpty#170, sum#171] -(116) CometHashAggregate -Input [3]: [sum#120, isEmpty#121, sum#122] +(133) HashAggregate [codegen id : 404] +Input [3]: [sum#169, isEmpty#170, sum#171] Keys: [] -Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] -Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] -Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] -Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] +Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] +Results [6]: [null AS channel#174, null AS i_brand_id#175, null AS i_class_id#176, null AS i_category_id#177, sum(sum_sales#164)#172 AS sum(sum_sales)#178, sum(number_sales#165)#173 AS sum(number_sales)#179] + +(134) Union + +(135) HashAggregate [codegen id : 405] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(136) CometColumnarExchange +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(137) CometHashAggregate +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] Functions: [] -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(138) CometTakeOrderedAndProject +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#108,number_sales#109]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109], 100, 0, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +(139) CometColumnarToRow [codegen id : 406] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] ===== Subqueries ===== -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] +Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (159) ++- * CometColumnarToRow (158) + +- CometColumnarExchange (157) + +- * HashAggregate (156) + +- Union (155) + :- * Project (144) + : +- * BroadcastHashJoin Inner BuildRight (143) + : :- * CometColumnarToRow (141) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (140) + : +- ReusedExchange (142) + :- * Project (149) + : +- * BroadcastHashJoin Inner BuildRight (148) + : :- * CometColumnarToRow (146) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (145) + : +- ReusedExchange (147) + +- * Project (154) + +- * BroadcastHashJoin Inner BuildRight (153) + :- * CometColumnarToRow (151) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (150) + +- ReusedExchange (152) + + +(140) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] +PartitionFilters: [isnotnull(ss_sold_date_sk#182), dynamicpruningexpression(ss_sold_date_sk#182 IN dynamicpruning#183)] ReadSchema: struct -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#147] +(141) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Right output [1]: [d_date_sk#147] -Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight +(142) ReusedExchange [Reuses operator id: 174] +Output [1]: [d_date_sk#184] -(126) CometProject -Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] -Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] +(143) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#182] +Right keys [1]: [d_date_sk#184] +Join type: Inner +Join condition: None -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] +(144) Project [codegen id : 2] +Output [2]: [ss_quantity#180 AS quantity#185, ss_list_price#181 AS list_price#186] +Input [4]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182, d_date_sk#184] + +(145) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] +PartitionFilters: [isnotnull(cs_sold_date_sk#189), dynamicpruningexpression(cs_sold_date_sk#189 IN dynamicpruning#190)] ReadSchema: struct -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) +(146) CometColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] -(130) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] +(147) ReusedExchange [Reuses operator id: 164] +Output [1]: [d_date_sk#191] -(131) CometBroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: [d_date_sk#154] +(148) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#189] +Right keys [1]: [d_date_sk#191] +Join type: Inner +Join condition: None -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Right output [1]: [d_date_sk#154] -Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight +(149) Project [codegen id : 4] +Output [2]: [cs_quantity#187 AS quantity#192, cs_list_price#188 AS list_price#193] +Input [4]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189, d_date_sk#191] -(133) CometProject -Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] -Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] +(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] +PartitionFilters: [isnotnull(ws_sold_date_sk#196), dynamicpruningexpression(ws_sold_date_sk#196 IN dynamicpruning#197)] ReadSchema: struct -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#162] +(151) CometColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] + +(152) ReusedExchange [Reuses operator id: 164] +Output [1]: [d_date_sk#198] -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Right output [1]: [d_date_sk#162] -Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight +(153) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#196] +Right keys [1]: [d_date_sk#198] +Join type: Inner +Join condition: None -(137) CometProject -Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] -Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] +(154) Project [codegen id : 6] +Output [2]: [ws_quantity#194 AS quantity#199, ws_list_price#195 AS list_price#200] +Input [4]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196, d_date_sk#198] -(138) CometUnion -Child 0 Input [2]: [quantity#148, list_price#149] -Child 1 Input [2]: [quantity#156, list_price#157] -Child 2 Input [2]: [quantity#163, list_price#164] +(155) Union -(139) CometHashAggregate -Input [2]: [quantity#148, list_price#149] +(156) HashAggregate [codegen id : 7] +Input [2]: [quantity#185, list_price#186] Keys: [] -Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] +Functions [1]: [partial_avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] +Aggregate Attributes [2]: [sum#201, count#202] +Results [2]: [sum#203, count#204] -(140) CometExchange -Input [2]: [sum#165, count#166] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(157) CometColumnarExchange +Input [2]: [sum#203, count#204] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(141) CometHashAggregate -Input [2]: [sum#165, count#166] -Keys: [] -Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] +(158) CometColumnarToRow [codegen id : 8] +Input [2]: [sum#203, count#204] -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#167] +(159) HashAggregate [codegen id : 8] +Input [2]: [sum#203, count#204] +Keys: [] +Functions [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] +Aggregate Attributes [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205] +Results [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205 AS average_sales#206] -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#182 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) +Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#189 IN dynamicpruning#190 +BroadcastExchange (164) ++- * CometColumnarToRow (163) + +- CometProject (162) + +- CometFilter (161) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (160) -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] +(160) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#191, d_year#207] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(144) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) +(161) CometFilter +Input [2]: [d_date_sk#191, d_year#207] +Condition : (((isnotnull(d_year#207) AND (d_year#207 >= 1998)) AND (d_year#207 <= 2000)) AND isnotnull(d_date_sk#191)) -(145) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] +(162) CometProject +Input [2]: [d_date_sk#191, d_year#207] +Arguments: [d_date_sk#191], [d_date_sk#191] -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#154] +(163) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#191] -(147) BroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(164) BroadcastExchange +Input [1]: [d_date_sk#191] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 +Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#196 IN dynamicpruning#190 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) +BroadcastExchange (169) ++- * CometColumnarToRow (168) + +- CometProject (167) + +- CometFilter (166) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (165) -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +(165) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#42, d_year#208, d_moy#209] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) +(166) CometFilter +Input [3]: [d_date_sk#42, d_year#208, d_moy#209] +Condition : ((((isnotnull(d_year#208) AND isnotnull(d_moy#209)) AND (d_year#208 = 2000)) AND (d_moy#209 = 11)) AND isnotnull(d_date_sk#42)) -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] +(167) CometProject +Input [3]: [d_date_sk#42, d_year#208, d_moy#209] +Arguments: [d_date_sk#42], [d_date_sk#42] -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] +(168) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(169) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (174) ++- * CometColumnarToRow (173) + +- CometProject (172) + +- CometFilter (171) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (170) -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#168] +(170) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#210] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#168] -Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) +(171) CometFilter +Input [2]: [d_date_sk#25, d_year#210] +Condition : (((isnotnull(d_year#210) AND (d_year#210 >= 1999)) AND (d_year#210 <= 2001)) AND isnotnull(d_date_sk#25)) -(155) CometProject -Input [2]: [d_date_sk#26, d_year#168] -Arguments: [d_date_sk#26], [d_date_sk#26] +(172) CometProject +Input [2]: [d_date_sk#25, d_year#210] +Arguments: [d_date_sk#25], [d_date_sk#25] -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] +(173) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(174) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] +Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt index 83deb45432..13f50df6c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt @@ -1,2350 +1,2779 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 1067 out of 2302 eligible operators (46%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index 758e917335..b3f0140800 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -1,179 +1,276 @@ -WholeStageCodegen (1) +WholeStageCodegen (406) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk] #12 - CometProject [cs_quantity,cs_list_price] [quantity,list_price] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - CometHashAggregate [brand_id,class_id,category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] - CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (405) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (79) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (26) + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (52) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (78) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (161) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (160) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (242) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id] #19 + WholeStageCodegen (241) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (323) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #20 + WholeStageCodegen (322) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (404) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange #21 + WholeStageCodegen (403) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt index 72e9bd1b4a..cb77e85abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt @@ -1,150 +1,167 @@ == Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) +* CometColumnarToRow (163) ++- CometTakeOrderedAndProject (162) + +- RowToColumnar (161) + +- Union (160) + :- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (30) + : : : +- * BroadcastHashJoin Inner BuildRight (29) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * CometColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * CometColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (28) + : : : +- * CometColumnarToRow (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (31) + : +- BroadcastExchange (38) + : +- * CometColumnarToRow (37) + : +- CometProject (36) + : +- CometFilter (35) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) + :- * HashAggregate (73) + : +- * CometColumnarToRow (72) + : +- CometColumnarExchange (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (66) + : : +- * BroadcastHashJoin Inner BuildRight (65) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * Project (56) + : : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : : :- * Project (53) + : : : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : : : :- * Project (50) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : : :- * CometColumnarToRow (47) + : : : : : : : +- CometFilter (46) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (45) + : : : : : : +- ReusedExchange (48) + : : : : : +- ReusedExchange (51) + : : : : +- ReusedExchange (54) + : : : +- BroadcastExchange (61) + : : : +- * CometColumnarToRow (60) + : : : +- CometProject (59) + : : : +- CometFilter (58) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (57) + : : +- ReusedExchange (64) + : +- ReusedExchange (67) + :- * HashAggregate (102) + : +- * CometColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * Project (85) + : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : :- * Project (82) + : : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * CometColumnarToRow (76) + : : : : : : : +- CometFilter (75) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (74) + : : : : : : +- ReusedExchange (77) + : : : : : +- ReusedExchange (80) + : : : : +- ReusedExchange (83) + : : : +- BroadcastExchange (90) + : : : +- * CometColumnarToRow (89) + : : : +- CometProject (88) + : : : +- CometFilter (87) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (86) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (131) + : +- * CometColumnarToRow (130) + : +- CometColumnarExchange (129) + : +- * HashAggregate (128) + : +- * Project (127) + : +- * BroadcastHashJoin Inner BuildRight (126) + : :- * Project (124) + : : +- * BroadcastHashJoin Inner BuildRight (123) + : : :- * Project (121) + : : : +- * BroadcastHashJoin Inner BuildRight (120) + : : : :- * Project (114) + : : : : +- * BroadcastHashJoin Inner BuildRight (113) + : : : : :- * Project (111) + : : : : : +- * BroadcastHashJoin Inner BuildRight (110) + : : : : : :- * Project (108) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : : : :- * CometColumnarToRow (105) + : : : : : : : +- CometFilter (104) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (103) + : : : : : : +- ReusedExchange (106) + : : : : : +- ReusedExchange (109) + : : : : +- ReusedExchange (112) + : : : +- BroadcastExchange (119) + : : : +- * CometColumnarToRow (118) + : : : +- CometProject (117) + : : : +- CometFilter (116) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (115) + : : +- ReusedExchange (122) + : +- ReusedExchange (125) + +- * HashAggregate (159) + +- * CometColumnarToRow (158) + +- CometColumnarExchange (157) + +- * HashAggregate (156) + +- * Project (155) + +- * BroadcastHashJoin Inner BuildRight (154) + :- * Project (149) + : +- * BroadcastHashJoin Inner BuildRight (148) + : :- * Project (146) + : : +- * BroadcastHashJoin Inner BuildRight (145) + : : :- * Project (143) + : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : :- * Project (140) + : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : :- * Project (137) + : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : :- * CometColumnarToRow (134) + : : : : : : +- CometFilter (133) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (132) + : : : : : +- ReusedExchange (135) + : : : : +- ReusedExchange (138) + : : : +- ReusedExchange (141) + : : +- ReusedExchange (144) + : +- ReusedExchange (147) + +- BroadcastExchange (153) + +- * CometColumnarToRow (152) + +- CometFilter (151) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (150) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -159,688 +176,777 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) -(5) CometProject +(6) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight +(8) BroadcastExchange +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 7] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(10) CometFilter +(12) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(11) CometProject +(13) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(12) CometBroadcastExchange +(14) CometColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight +(16) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None -(14) CometProject +(17) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(16) CometFilter +(19) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(17) CometBroadcastExchange +(20) CometColumnarToRow [codegen id : 3] +Input [1]: [cd_demo_sk#20] + +(21) BroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight +(22) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#16] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None -(19) CometProject +(23) Project [codegen id : 7] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(21) CometFilter +(25) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(22) CometProject +(26) CometProject Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(23) CometBroadcastExchange +(27) CometColumnarToRow [codegen id : 4] Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(28) BroadcastExchange +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#17] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] +(30) Project [codegen id : 7] +Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] +(31) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#26] +Join type: Inner +Join condition: None -(31) CometProject +(33) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +(35) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) + +(36) CometProject +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] + +(37) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_sk#27, i_item_id#29] + +(38) BroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(39) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#27] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 7] +Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] + +(41) HashAggregate [codegen id : 7] +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] +Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] +Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] +Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] +Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] + +(42) CometColumnarExchange +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(43) CometColumnarToRow [codegen id : 8] +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] + +(44) HashAggregate [codegen id : 8] +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] +Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] +Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] +Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] + +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) +(46) CometFilter +Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] +(47) CometColumnarToRow [codegen id : 15] +Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight +(48) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#93, cd_dep_count#94] -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] +(49) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_cdemo_sk#84] +Right keys [1]: [cd_demo_sk#93] +Join type: Inner +Join condition: None -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +(50) Project [codegen id : 15] +Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94] +Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#93, cd_dep_count#94] -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight +(51) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +(52) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_customer_sk#83] +Right keys [1]: [c_customer_sk#95] +Join type: Inner +Join condition: None -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] +(53) Project [codegen id : 15] +Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight +(54) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#99] -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] +(55) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#99] +Join type: Inner +Join condition: None -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +(56) Project [codegen id : 15] +Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98, cd_demo_sk#99] + +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +(58) CometFilter +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) + +(59) CometProject +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: [ca_address_sk#100, ca_state#25, ca_country#102], [ca_address_sk#100, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) AS ca_state#25, ca_country#102] + +(60) CometColumnarToRow [codegen id : 12] +Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] + +(61) BroadcastExchange +Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(62) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 15] +Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] +Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98, ca_address_sk#100, ca_state#25, ca_country#102] + +(64) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#103] + +(65) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#91] +Right keys [1]: [d_date_sk#103] +Join type: Inner +Join condition: None + +(66) Project [codegen id : 15] +Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, d_date_sk#103] + +(67) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#104, i_item_id#29] + +(68) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_item_sk#85] +Right keys [1]: [i_item_sk#104] +Join type: Inner +Join condition: None + +(69) Project [codegen id : 15] +Output [10]: [i_item_id#29, ca_country#102, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#105, cast(cs_list_price#87 as decimal(12,2)) AS agg2#106, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#107, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#108, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#109, cast(c_birth_year#98 as decimal(12,2)) AS agg6#110, cast(cd_dep_count#94 as decimal(12,2)) AS agg7#111] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, i_item_sk#104, i_item_id#29] + +(70) HashAggregate [codegen id : 15] +Input [10]: [i_item_id#29, ca_country#102, ca_state#25, agg1#105, agg2#106, agg3#107, agg4#108, agg5#109, agg6#110, agg7#111] +Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] +Functions [7]: [partial_avg(agg1#105), partial_avg(agg2#106), partial_avg(agg3#107), partial_avg(agg4#108), partial_avg(agg5#109), partial_avg(agg6#110), partial_avg(agg7#111)] +Aggregate Attributes [14]: [sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +Results [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] + +(71) CometColumnarExchange +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Arguments: hashpartitioning(i_item_id#29, ca_country#102, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(72) CometColumnarToRow [codegen id : 16] +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] + +(73) HashAggregate [codegen id : 16] +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] +Functions [7]: [avg(agg1#105), avg(agg2#106), avg(agg3#107), avg(agg4#108), avg(agg5#109), avg(agg6#110), avg(agg7#111)] +Aggregate Attributes [7]: [avg(agg1#105)#140, avg(agg2#106)#141, avg(agg3#107)#142, avg(agg4#108)#143, avg(agg5#109)#144, avg(agg6#110)#145, avg(agg7#111)#146] +Results [11]: [i_item_id#29, ca_country#102, ca_state#25, null AS county#147, avg(agg1#105)#140 AS agg1#148, avg(agg2#106)#141 AS agg2#149, avg(agg3#107)#142 AS agg3#150, avg(agg4#108)#143 AS agg4#151, avg(agg5#109)#144 AS agg5#152, avg(agg6#110)#145 AS agg6#153, avg(agg7#111)#146 AS agg7#154] + +(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] +PartitionFilters: [isnotnull(cs_sold_date_sk#163), dynamicpruningexpression(cs_sold_date_sk#163 IN dynamicpruning#164)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) +(75) CometFilter +Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +Condition : ((isnotnull(cs_bill_cdemo_sk#156) AND isnotnull(cs_bill_customer_sk#155)) AND isnotnull(cs_item_sk#157)) + +(76) CometColumnarToRow [codegen id : 23] +Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#109, cd_dep_count#110] +(77) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#165, cd_dep_count#166] -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Right output [2]: [cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight +(78) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_cdemo_sk#156] +Right keys [1]: [cd_demo_sk#165] +Join type: Inner +Join condition: None -(71) CometProject -Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] +(79) Project [codegen id : 23] +Output [9]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166] +Input [11]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_demo_sk#165, cd_dep_count#166] -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +(80) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight +(81) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_customer_sk#155] +Right keys [1]: [c_customer_sk#167] +Join type: Inner +Join condition: None -(74) CometProject -Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +(82) Project [codegen id : 23] +Output [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +Input [13]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#115] +(83) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#171] -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Right output [1]: [cd_demo_sk#115] -Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight +(84) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_cdemo_sk#168] +Right keys [1]: [cd_demo_sk#171] +Join type: Inner +Join condition: None -(77) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] +(85) Project [codegen id : 23] +Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] +Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#172, ca_state#173, ca_country#174] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(79) CometFilter -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) - -(80) CometProject -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#116, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -Right output [2]: [ca_address_sk#116, ca_country#118] -Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#119] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [1]: [d_date_sk#119] -Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#120, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [2]: [i_item_sk#120, i_item_id#30] -Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] -Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +(87) CometFilter +Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#173, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#172)) + +(88) CometProject +Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] + +(89) CometColumnarToRow [codegen id : 20] +Input [2]: [ca_address_sk#172, ca_country#174] + +(90) BroadcastExchange +Input [2]: [ca_address_sk#172, ca_country#174] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(91) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_addr_sk#169] +Right keys [1]: [ca_address_sk#172] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 23] +Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174] +Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] + +(93) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#175] + +(94) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#163] +Right keys [1]: [d_date_sk#175] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 23] +Output [9]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174] +Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] + +(96) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#176, i_item_id#29] + +(97) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#157] +Right keys [1]: [i_item_sk#176] +Join type: Inner +Join condition: None + +(98) Project [codegen id : 23] +Output [9]: [i_item_id#29, ca_country#174, cast(cs_quantity#158 as decimal(12,2)) AS agg1#177, cast(cs_list_price#159 as decimal(12,2)) AS agg2#178, cast(cs_coupon_amt#161 as decimal(12,2)) AS agg3#179, cast(cs_sales_price#160 as decimal(12,2)) AS agg4#180, cast(cs_net_profit#162 as decimal(12,2)) AS agg5#181, cast(c_birth_year#170 as decimal(12,2)) AS agg6#182, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#183] +Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#29] + +(99) HashAggregate [codegen id : 23] +Input [9]: [i_item_id#29, ca_country#174, agg1#177, agg2#178, agg3#179, agg4#180, agg5#181, agg6#182, agg7#183] +Keys [2]: [i_item_id#29, ca_country#174] +Functions [7]: [partial_avg(agg1#177), partial_avg(agg2#178), partial_avg(agg3#179), partial_avg(agg4#180), partial_avg(agg5#181), partial_avg(agg6#182), partial_avg(agg7#183)] +Aggregate Attributes [14]: [sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197] +Results [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] + +(100) CometColumnarExchange +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] +Arguments: hashpartitioning(i_item_id#29, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(101) CometColumnarToRow [codegen id : 24] +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] + +(102) HashAggregate [codegen id : 24] +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] +Keys [2]: [i_item_id#29, ca_country#174] +Functions [7]: [avg(agg1#177), avg(agg2#178), avg(agg3#179), avg(agg4#180), avg(agg5#181), avg(agg6#182), avg(agg7#183)] +Aggregate Attributes [7]: [avg(agg1#177)#212, avg(agg2#178)#213, avg(agg3#179)#214, avg(agg4#180)#215, avg(agg5#181)#216, avg(agg6#182)#217, avg(agg7#183)#218] +Results [11]: [i_item_id#29, ca_country#174, null AS ca_state#219, null AS county#220, avg(agg1#177)#212 AS agg1#221, avg(agg2#178)#213 AS agg2#222, avg(agg3#179)#214 AS agg3#223, avg(agg4#180)#215 AS agg4#224, avg(agg5#181)#216 AS agg5#225, avg(agg6#182)#217 AS agg6#226, avg(agg7#183)#218 AS agg7#227] + +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] +PartitionFilters: [isnotnull(cs_sold_date_sk#236), dynamicpruningexpression(cs_sold_date_sk#236 IN dynamicpruning#237)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(94) CometFilter -Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) +(104) CometFilter +Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +Condition : ((isnotnull(cs_bill_cdemo_sk#229) AND isnotnull(cs_bill_customer_sk#228)) AND isnotnull(cs_item_sk#230)) -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#152, cd_dep_count#153] +(105) CometColumnarToRow [codegen id : 31] +Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Right output [2]: [cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight +(106) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#238, cd_dep_count#239] -(97) CometProject -Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] +(107) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#238] +Join type: Inner +Join condition: None -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +(108) Project [codegen id : 31] +Output [9]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239] +Input [11]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_demo_sk#238, cd_dep_count#239] -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight +(109) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -(100) CometProject -Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +(110) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_customer_sk#228] +Right keys [1]: [c_customer_sk#240] +Join type: Inner +Join condition: None -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#158] +(111) Project [codegen id : 31] +Output [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +Input [13]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [cd_demo_sk#158] -Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight +(112) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#244] -(103) CometProject -Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] +(113) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_cdemo_sk#241] +Right keys [1]: [cd_demo_sk#244] +Join type: Inner +Join condition: None -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#159, ca_state#160] +(114) Project [codegen id : 31] +Output [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] +Input [12]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] + +(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#245, ca_state#246] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [ca_address_sk#159, ca_state#160] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) - -(106) CometProject -Input [2]: [ca_address_sk#159, ca_state#160] -Arguments: [ca_address_sk#159], [ca_address_sk#159] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#159] -Arguments: [ca_address_sk#159] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [ca_address_sk#159] -Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#161] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] -Right output [1]: [d_date_sk#161] -Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#162, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] -Right output [2]: [i_item_sk#162, i_item_id#30] -Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] -Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +(116) CometFilter +Input [2]: [ca_address_sk#245, ca_state#246] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#246, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#245)) + +(117) CometProject +Input [2]: [ca_address_sk#245, ca_state#246] +Arguments: [ca_address_sk#245], [ca_address_sk#245] + +(118) CometColumnarToRow [codegen id : 28] +Input [1]: [ca_address_sk#245] + +(119) BroadcastExchange +Input [1]: [ca_address_sk#245] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(120) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_addr_sk#242] +Right keys [1]: [ca_address_sk#245] +Join type: Inner +Join condition: None + +(121) Project [codegen id : 31] +Output [9]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243] +Input [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] + +(122) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#247] + +(123) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_sold_date_sk#236] +Right keys [1]: [d_date_sk#247] +Join type: Inner +Join condition: None + +(124) Project [codegen id : 31] +Output [8]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243] +Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243, d_date_sk#247] + +(125) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#248, i_item_id#29] + +(126) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_item_sk#230] +Right keys [1]: [i_item_sk#248] +Join type: Inner +Join condition: None + +(127) Project [codegen id : 31] +Output [8]: [i_item_id#29, cast(cs_quantity#231 as decimal(12,2)) AS agg1#249, cast(cs_list_price#232 as decimal(12,2)) AS agg2#250, cast(cs_coupon_amt#234 as decimal(12,2)) AS agg3#251, cast(cs_sales_price#233 as decimal(12,2)) AS agg4#252, cast(cs_net_profit#235 as decimal(12,2)) AS agg5#253, cast(c_birth_year#243 as decimal(12,2)) AS agg6#254, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#255] +Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#29] + +(128) HashAggregate [codegen id : 31] +Input [8]: [i_item_id#29, agg1#249, agg2#250, agg3#251, agg4#252, agg5#253, agg6#254, agg7#255] +Keys [1]: [i_item_id#29] +Functions [7]: [partial_avg(agg1#249), partial_avg(agg2#250), partial_avg(agg3#251), partial_avg(agg4#252), partial_avg(agg5#253), partial_avg(agg6#254), partial_avg(agg7#255)] +Aggregate Attributes [14]: [sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Results [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] + +(129) CometColumnarExchange +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(130) CometColumnarToRow [codegen id : 32] +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] + +(131) HashAggregate [codegen id : 32] +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +Keys [1]: [i_item_id#29] +Functions [7]: [avg(agg1#249), avg(agg2#250), avg(agg3#251), avg(agg4#252), avg(agg5#253), avg(agg6#254), avg(agg7#255)] +Aggregate Attributes [7]: [avg(agg1#249)#284, avg(agg2#250)#285, avg(agg3#251)#286, avg(agg4#252)#287, avg(agg5#253)#288, avg(agg6#254)#289, avg(agg7#255)#290] +Results [11]: [i_item_id#29, null AS ca_country#291, null AS ca_state#292, null AS county#293, avg(agg1#249)#284 AS agg1#294, avg(agg2#250)#285 AS agg2#295, avg(agg3#251)#286 AS agg3#296, avg(agg4#252)#287 AS agg4#297, avg(agg5#253)#288 AS agg5#298, avg(agg6#254)#289 AS agg6#299, avg(agg7#255)#290 AS agg7#300] + +(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] +PartitionFilters: [isnotnull(cs_sold_date_sk#309), dynamicpruningexpression(cs_sold_date_sk#309 IN dynamicpruning#310)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(120) CometFilter -Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) +(133) CometFilter +Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +Condition : ((isnotnull(cs_bill_cdemo_sk#302) AND isnotnull(cs_bill_customer_sk#301)) AND isnotnull(cs_item_sk#303)) -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#194, cd_dep_count#195] +(134) CometColumnarToRow [codegen id : 39] +Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Right output [2]: [cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight +(135) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#311, cd_dep_count#312] -(123) CometProject -Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] +(136) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_cdemo_sk#302] +Right keys [1]: [cd_demo_sk#311] +Join type: Inner +Join condition: None -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +(137) Project [codegen id : 39] +Output [9]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312] +Input [11]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_demo_sk#311, cd_dep_count#312] -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight +(138) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -(126) CometProject -Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +(139) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_customer_sk#301] +Right keys [1]: [c_customer_sk#313] +Join type: Inner +Join condition: None -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#200] +(140) Project [codegen id : 39] +Output [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +Input [13]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [cd_demo_sk#200] -Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight +(141) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#317] -(129) CometProject -Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] +(142) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_cdemo_sk#314] +Right keys [1]: [cd_demo_sk#317] +Join type: Inner +Join condition: None -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#201] +(143) Project [codegen id : 39] +Output [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] +Input [12]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [ca_address_sk#201] -Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight +(144) ReusedExchange [Reuses operator id: 119] +Output [1]: [ca_address_sk#318] -(132) CometProject -Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] +(145) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_addr_sk#315] +Right keys [1]: [ca_address_sk#318] +Join type: Inner +Join condition: None -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#202] +(146) Project [codegen id : 39] +Output [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316] +Input [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -Right output [1]: [d_date_sk#202] -Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight +(147) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#319] -(135) CometProject -Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] +(148) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_sold_date_sk#309] +Right keys [1]: [d_date_sk#319] +Join type: Inner +Join condition: None -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#203] +(149) Project [codegen id : 39] +Output [8]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316] +Input [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316, d_date_sk#319] + +(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#320] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(137) CometFilter -Input [1]: [i_item_sk#203] -Condition : isnotnull(i_item_sk#203) +(151) CometFilter +Input [1]: [i_item_sk#320] +Condition : isnotnull(i_item_sk#320) + +(152) CometColumnarToRow [codegen id : 38] +Input [1]: [i_item_sk#320] -(138) CometBroadcastExchange -Input [1]: [i_item_sk#203] -Arguments: [i_item_sk#203] +(153) BroadcastExchange +Input [1]: [i_item_sk#320] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -Right output [1]: [i_item_sk#203] -Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight +(154) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_item_sk#303] +Right keys [1]: [i_item_sk#320] +Join type: Inner +Join condition: None -(140) CometProject -Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] -Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] +(155) Project [codegen id : 39] +Output [7]: [cast(cs_quantity#304 as decimal(12,2)) AS agg1#321, cast(cs_list_price#305 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#307 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#306 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#308 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] +Input [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316, i_item_sk#320] -(141) CometHashAggregate -Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] +(156) HashAggregate [codegen id : 39] +Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] Keys: [] -Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] +Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] +Aggregate Attributes [14]: [sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339, sum#340, count#341] +Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -(142) CometExchange -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(157) CometColumnarExchange +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(143) CometHashAggregate -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] +(158) CometColumnarToRow [codegen id : 40] +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] + +(159) HashAggregate [codegen id : 40] +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] Keys: [] -Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] +Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] +Aggregate Attributes [7]: [avg(agg1#321)#356, avg(agg2#322)#357, avg(agg3#323)#358, avg(agg4#324)#359, avg(agg5#325)#360, avg(agg6#326)#361, avg(agg7#327)#362] +Results [11]: [null AS i_item_id#363, null AS ca_country#364, null AS ca_state#365, null AS county#366, avg(agg1#321)#356 AS agg1#367, avg(agg2#322)#357 AS agg2#368, avg(agg3#323)#358 AS agg3#369, avg(agg4#324)#359 AS agg4#370, avg(agg5#325)#360 AS agg5#371, avg(agg6#326)#361 AS agg6#372, avg(agg7#327)#362 AS agg7#373] + +(160) Union -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] -Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] -Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] -Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] +(161) RowToColumnar +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +(162) CometTakeOrderedAndProject +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#31 ASC NULLS FIRST,ca_state#32 ASC NULLS FIRST,ca_county#33 ASC NULLS FIRST,i_item_id#30 ASC NULLS FIRST], output=[i_item_id#30,ca_country#31,ca_state#32,ca_county#33,agg1#76,agg2#77,agg3#78,agg4#79,agg5#80,agg6#81,agg7#82]), [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82], 100, 0, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +(163) CometColumnarToRow [codegen id : 41] +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) +BroadcastExchange (168) ++- * CometColumnarToRow (167) + +- CometProject (166) + +- CometFilter (165) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (164) -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(164) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#374] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) +(165) CometFilter +Input [2]: [d_date_sk#26, d_year#374] +Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_date_sk#26)) -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(166) CometProject +Input [2]: [d_date_sk#26, d_year#374] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(167) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(151) BroadcastExchange +(168) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#163 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#236 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#309 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index b18a444bb9..7a8f23c282 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -1,214 +1,255 @@ CometColumnarToRow +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 103 out of 210 eligible operators (49%). Final plan contains 43 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt index 1572a2a240..91800fcca0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt @@ -1,160 +1,245 @@ -WholeStageCodegen (1) +WholeStageCodegen (41) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (16) + HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state] #8 + WholeStageCodegen (15) + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (24) + HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country] #10 + WholeStageCodegen (23) + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (32) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #12 + WholeStageCodegen (31) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (28) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (40) + HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (39) + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + ReusedExchange [ca_address_sk] #13 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (38) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt index 72e9bd1b4a..cb77e85abc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt @@ -1,150 +1,167 @@ == Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) +* CometColumnarToRow (163) ++- CometTakeOrderedAndProject (162) + +- RowToColumnar (161) + +- Union (160) + :- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (30) + : : : +- * BroadcastHashJoin Inner BuildRight (29) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * CometColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * CometColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (28) + : : : +- * CometColumnarToRow (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (31) + : +- BroadcastExchange (38) + : +- * CometColumnarToRow (37) + : +- CometProject (36) + : +- CometFilter (35) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) + :- * HashAggregate (73) + : +- * CometColumnarToRow (72) + : +- CometColumnarExchange (71) + : +- * HashAggregate (70) + : +- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) + : :- * Project (66) + : : +- * BroadcastHashJoin Inner BuildRight (65) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * Project (56) + : : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : : :- * Project (53) + : : : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : : : :- * Project (50) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : : :- * CometColumnarToRow (47) + : : : : : : : +- CometFilter (46) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (45) + : : : : : : +- ReusedExchange (48) + : : : : : +- ReusedExchange (51) + : : : : +- ReusedExchange (54) + : : : +- BroadcastExchange (61) + : : : +- * CometColumnarToRow (60) + : : : +- CometProject (59) + : : : +- CometFilter (58) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (57) + : : +- ReusedExchange (64) + : +- ReusedExchange (67) + :- * HashAggregate (102) + : +- * CometColumnarToRow (101) + : +- CometColumnarExchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * Project (85) + : : : : +- * BroadcastHashJoin Inner BuildRight (84) + : : : : :- * Project (82) + : : : : : +- * BroadcastHashJoin Inner BuildRight (81) + : : : : : :- * Project (79) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) + : : : : : : :- * CometColumnarToRow (76) + : : : : : : : +- CometFilter (75) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (74) + : : : : : : +- ReusedExchange (77) + : : : : : +- ReusedExchange (80) + : : : : +- ReusedExchange (83) + : : : +- BroadcastExchange (90) + : : : +- * CometColumnarToRow (89) + : : : +- CometProject (88) + : : : +- CometFilter (87) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (86) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (131) + : +- * CometColumnarToRow (130) + : +- CometColumnarExchange (129) + : +- * HashAggregate (128) + : +- * Project (127) + : +- * BroadcastHashJoin Inner BuildRight (126) + : :- * Project (124) + : : +- * BroadcastHashJoin Inner BuildRight (123) + : : :- * Project (121) + : : : +- * BroadcastHashJoin Inner BuildRight (120) + : : : :- * Project (114) + : : : : +- * BroadcastHashJoin Inner BuildRight (113) + : : : : :- * Project (111) + : : : : : +- * BroadcastHashJoin Inner BuildRight (110) + : : : : : :- * Project (108) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : : : :- * CometColumnarToRow (105) + : : : : : : : +- CometFilter (104) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (103) + : : : : : : +- ReusedExchange (106) + : : : : : +- ReusedExchange (109) + : : : : +- ReusedExchange (112) + : : : +- BroadcastExchange (119) + : : : +- * CometColumnarToRow (118) + : : : +- CometProject (117) + : : : +- CometFilter (116) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (115) + : : +- ReusedExchange (122) + : +- ReusedExchange (125) + +- * HashAggregate (159) + +- * CometColumnarToRow (158) + +- CometColumnarExchange (157) + +- * HashAggregate (156) + +- * Project (155) + +- * BroadcastHashJoin Inner BuildRight (154) + :- * Project (149) + : +- * BroadcastHashJoin Inner BuildRight (148) + : :- * Project (146) + : : +- * BroadcastHashJoin Inner BuildRight (145) + : : :- * Project (143) + : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : :- * Project (140) + : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : :- * Project (137) + : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : :- * CometColumnarToRow (134) + : : : : : : +- CometFilter (133) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (132) + : : : : : +- ReusedExchange (135) + : : : : +- ReusedExchange (138) + : : : +- ReusedExchange (141) + : : +- ReusedExchange (144) + : +- ReusedExchange (147) + +- BroadcastExchange (153) + +- * CometColumnarToRow (152) + +- CometFilter (151) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (150) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -159,688 +176,777 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) -(5) CometProject +(6) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight +(8) BroadcastExchange +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 7] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(10) CometFilter +(12) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(11) CometProject +(13) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(12) CometBroadcastExchange +(14) CometColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight +(16) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None -(14) CometProject +(17) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(16) CometFilter +(19) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(17) CometBroadcastExchange +(20) CometColumnarToRow [codegen id : 3] +Input [1]: [cd_demo_sk#20] + +(21) BroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight +(22) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#16] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None -(19) CometProject +(23) Project [codegen id : 7] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(21) CometFilter +(25) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(22) CometProject +(26) CometProject Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(23) CometBroadcastExchange +(27) CometColumnarToRow [codegen id : 4] Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +(28) BroadcastExchange +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#17] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] +(30) Project [codegen id : 7] +Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] +(31) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#26] -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#26] +Join type: Inner +Join condition: None -(31) CometProject +(33) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +(35) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) + +(36) CometProject +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] + +(37) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_sk#27, i_item_id#29] + +(38) BroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(39) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#27] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 7] +Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] + +(41) HashAggregate [codegen id : 7] +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] +Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] +Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] +Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] +Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] + +(42) CometColumnarExchange +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(43) CometColumnarToRow [codegen id : 8] +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] + +(44) HashAggregate [codegen id : 8] +Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] +Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] +Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] +Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] + +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) +(46) CometFilter +Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] +(47) CometColumnarToRow [codegen id : 15] +Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight +(48) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#93, cd_dep_count#94] -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] +(49) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_cdemo_sk#84] +Right keys [1]: [cd_demo_sk#93] +Join type: Inner +Join condition: None -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +(50) Project [codegen id : 15] +Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94] +Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#93, cd_dep_count#94] -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight +(51) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +(52) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_customer_sk#83] +Right keys [1]: [c_customer_sk#95] +Join type: Inner +Join condition: None -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] +(53) Project [codegen id : 15] +Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight +(54) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#99] -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] +(55) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#99] +Join type: Inner +Join condition: None -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +(56) Project [codegen id : 15] +Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98, cd_demo_sk#99] + +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +(58) CometFilter +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) + +(59) CometProject +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: [ca_address_sk#100, ca_state#25, ca_country#102], [ca_address_sk#100, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) AS ca_state#25, ca_country#102] + +(60) CometColumnarToRow [codegen id : 12] +Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] + +(61) BroadcastExchange +Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(62) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 15] +Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] +Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98, ca_address_sk#100, ca_state#25, ca_country#102] + +(64) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#103] + +(65) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#91] +Right keys [1]: [d_date_sk#103] +Join type: Inner +Join condition: None + +(66) Project [codegen id : 15] +Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, d_date_sk#103] + +(67) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#104, i_item_id#29] + +(68) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_item_sk#85] +Right keys [1]: [i_item_sk#104] +Join type: Inner +Join condition: None + +(69) Project [codegen id : 15] +Output [10]: [i_item_id#29, ca_country#102, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#105, cast(cs_list_price#87 as decimal(12,2)) AS agg2#106, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#107, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#108, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#109, cast(c_birth_year#98 as decimal(12,2)) AS agg6#110, cast(cd_dep_count#94 as decimal(12,2)) AS agg7#111] +Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, i_item_sk#104, i_item_id#29] + +(70) HashAggregate [codegen id : 15] +Input [10]: [i_item_id#29, ca_country#102, ca_state#25, agg1#105, agg2#106, agg3#107, agg4#108, agg5#109, agg6#110, agg7#111] +Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] +Functions [7]: [partial_avg(agg1#105), partial_avg(agg2#106), partial_avg(agg3#107), partial_avg(agg4#108), partial_avg(agg5#109), partial_avg(agg6#110), partial_avg(agg7#111)] +Aggregate Attributes [14]: [sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +Results [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] + +(71) CometColumnarExchange +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Arguments: hashpartitioning(i_item_id#29, ca_country#102, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(72) CometColumnarToRow [codegen id : 16] +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] + +(73) HashAggregate [codegen id : 16] +Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] +Functions [7]: [avg(agg1#105), avg(agg2#106), avg(agg3#107), avg(agg4#108), avg(agg5#109), avg(agg6#110), avg(agg7#111)] +Aggregate Attributes [7]: [avg(agg1#105)#140, avg(agg2#106)#141, avg(agg3#107)#142, avg(agg4#108)#143, avg(agg5#109)#144, avg(agg6#110)#145, avg(agg7#111)#146] +Results [11]: [i_item_id#29, ca_country#102, ca_state#25, null AS county#147, avg(agg1#105)#140 AS agg1#148, avg(agg2#106)#141 AS agg2#149, avg(agg3#107)#142 AS agg3#150, avg(agg4#108)#143 AS agg4#151, avg(agg5#109)#144 AS agg5#152, avg(agg6#110)#145 AS agg6#153, avg(agg7#111)#146 AS agg7#154] + +(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] +PartitionFilters: [isnotnull(cs_sold_date_sk#163), dynamicpruningexpression(cs_sold_date_sk#163 IN dynamicpruning#164)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter -Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) +(75) CometFilter +Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +Condition : ((isnotnull(cs_bill_cdemo_sk#156) AND isnotnull(cs_bill_customer_sk#155)) AND isnotnull(cs_item_sk#157)) + +(76) CometColumnarToRow [codegen id : 23] +Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#109, cd_dep_count#110] +(77) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#165, cd_dep_count#166] -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Right output [2]: [cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight +(78) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_cdemo_sk#156] +Right keys [1]: [cd_demo_sk#165] +Join type: Inner +Join condition: None -(71) CometProject -Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] +(79) Project [codegen id : 23] +Output [9]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166] +Input [11]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_demo_sk#165, cd_dep_count#166] -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +(80) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight +(81) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_customer_sk#155] +Right keys [1]: [c_customer_sk#167] +Join type: Inner +Join condition: None -(74) CometProject -Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +(82) Project [codegen id : 23] +Output [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +Input [13]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#115] +(83) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#171] -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Right output [1]: [cd_demo_sk#115] -Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight +(84) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_cdemo_sk#168] +Right keys [1]: [cd_demo_sk#171] +Join type: Inner +Join condition: None -(77) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] +(85) Project [codegen id : 23] +Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] +Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#172, ca_state#173, ca_country#174] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(79) CometFilter -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) - -(80) CometProject -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#116, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -Right output [2]: [ca_address_sk#116, ca_country#118] -Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#119] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [1]: [d_date_sk#119] -Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#120, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [2]: [i_item_sk#120, i_item_id#30] -Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] -Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +(87) CometFilter +Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#173, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#172)) + +(88) CometProject +Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] + +(89) CometColumnarToRow [codegen id : 20] +Input [2]: [ca_address_sk#172, ca_country#174] + +(90) BroadcastExchange +Input [2]: [ca_address_sk#172, ca_country#174] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(91) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_addr_sk#169] +Right keys [1]: [ca_address_sk#172] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 23] +Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174] +Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] + +(93) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#175] + +(94) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#163] +Right keys [1]: [d_date_sk#175] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 23] +Output [9]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174] +Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] + +(96) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#176, i_item_id#29] + +(97) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#157] +Right keys [1]: [i_item_sk#176] +Join type: Inner +Join condition: None + +(98) Project [codegen id : 23] +Output [9]: [i_item_id#29, ca_country#174, cast(cs_quantity#158 as decimal(12,2)) AS agg1#177, cast(cs_list_price#159 as decimal(12,2)) AS agg2#178, cast(cs_coupon_amt#161 as decimal(12,2)) AS agg3#179, cast(cs_sales_price#160 as decimal(12,2)) AS agg4#180, cast(cs_net_profit#162 as decimal(12,2)) AS agg5#181, cast(c_birth_year#170 as decimal(12,2)) AS agg6#182, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#183] +Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#29] + +(99) HashAggregate [codegen id : 23] +Input [9]: [i_item_id#29, ca_country#174, agg1#177, agg2#178, agg3#179, agg4#180, agg5#181, agg6#182, agg7#183] +Keys [2]: [i_item_id#29, ca_country#174] +Functions [7]: [partial_avg(agg1#177), partial_avg(agg2#178), partial_avg(agg3#179), partial_avg(agg4#180), partial_avg(agg5#181), partial_avg(agg6#182), partial_avg(agg7#183)] +Aggregate Attributes [14]: [sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197] +Results [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] + +(100) CometColumnarExchange +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] +Arguments: hashpartitioning(i_item_id#29, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(101) CometColumnarToRow [codegen id : 24] +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] + +(102) HashAggregate [codegen id : 24] +Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] +Keys [2]: [i_item_id#29, ca_country#174] +Functions [7]: [avg(agg1#177), avg(agg2#178), avg(agg3#179), avg(agg4#180), avg(agg5#181), avg(agg6#182), avg(agg7#183)] +Aggregate Attributes [7]: [avg(agg1#177)#212, avg(agg2#178)#213, avg(agg3#179)#214, avg(agg4#180)#215, avg(agg5#181)#216, avg(agg6#182)#217, avg(agg7#183)#218] +Results [11]: [i_item_id#29, ca_country#174, null AS ca_state#219, null AS county#220, avg(agg1#177)#212 AS agg1#221, avg(agg2#178)#213 AS agg2#222, avg(agg3#179)#214 AS agg3#223, avg(agg4#180)#215 AS agg4#224, avg(agg5#181)#216 AS agg5#225, avg(agg6#182)#217 AS agg6#226, avg(agg7#183)#218 AS agg7#227] + +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] +PartitionFilters: [isnotnull(cs_sold_date_sk#236), dynamicpruningexpression(cs_sold_date_sk#236 IN dynamicpruning#237)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(94) CometFilter -Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) +(104) CometFilter +Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +Condition : ((isnotnull(cs_bill_cdemo_sk#229) AND isnotnull(cs_bill_customer_sk#228)) AND isnotnull(cs_item_sk#230)) -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#152, cd_dep_count#153] +(105) CometColumnarToRow [codegen id : 31] +Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Right output [2]: [cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight +(106) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#238, cd_dep_count#239] -(97) CometProject -Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] +(107) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#238] +Join type: Inner +Join condition: None -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +(108) Project [codegen id : 31] +Output [9]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239] +Input [11]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_demo_sk#238, cd_dep_count#239] -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight +(109) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -(100) CometProject -Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +(110) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_customer_sk#228] +Right keys [1]: [c_customer_sk#240] +Join type: Inner +Join condition: None -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#158] +(111) Project [codegen id : 31] +Output [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +Input [13]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [cd_demo_sk#158] -Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight +(112) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#244] -(103) CometProject -Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] +(113) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_cdemo_sk#241] +Right keys [1]: [cd_demo_sk#244] +Join type: Inner +Join condition: None -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#159, ca_state#160] +(114) Project [codegen id : 31] +Output [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] +Input [12]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] + +(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#245, ca_state#246] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [ca_address_sk#159, ca_state#160] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) - -(106) CometProject -Input [2]: [ca_address_sk#159, ca_state#160] -Arguments: [ca_address_sk#159], [ca_address_sk#159] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#159] -Arguments: [ca_address_sk#159] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [ca_address_sk#159] -Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#161] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] -Right output [1]: [d_date_sk#161] -Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#162, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] -Right output [2]: [i_item_sk#162, i_item_id#30] -Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] -Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +(116) CometFilter +Input [2]: [ca_address_sk#245, ca_state#246] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#246, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#245)) + +(117) CometProject +Input [2]: [ca_address_sk#245, ca_state#246] +Arguments: [ca_address_sk#245], [ca_address_sk#245] + +(118) CometColumnarToRow [codegen id : 28] +Input [1]: [ca_address_sk#245] + +(119) BroadcastExchange +Input [1]: [ca_address_sk#245] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(120) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_addr_sk#242] +Right keys [1]: [ca_address_sk#245] +Join type: Inner +Join condition: None + +(121) Project [codegen id : 31] +Output [9]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243] +Input [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] + +(122) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#247] + +(123) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_sold_date_sk#236] +Right keys [1]: [d_date_sk#247] +Join type: Inner +Join condition: None + +(124) Project [codegen id : 31] +Output [8]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243] +Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243, d_date_sk#247] + +(125) ReusedExchange [Reuses operator id: 38] +Output [2]: [i_item_sk#248, i_item_id#29] + +(126) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_item_sk#230] +Right keys [1]: [i_item_sk#248] +Join type: Inner +Join condition: None + +(127) Project [codegen id : 31] +Output [8]: [i_item_id#29, cast(cs_quantity#231 as decimal(12,2)) AS agg1#249, cast(cs_list_price#232 as decimal(12,2)) AS agg2#250, cast(cs_coupon_amt#234 as decimal(12,2)) AS agg3#251, cast(cs_sales_price#233 as decimal(12,2)) AS agg4#252, cast(cs_net_profit#235 as decimal(12,2)) AS agg5#253, cast(c_birth_year#243 as decimal(12,2)) AS agg6#254, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#255] +Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#29] + +(128) HashAggregate [codegen id : 31] +Input [8]: [i_item_id#29, agg1#249, agg2#250, agg3#251, agg4#252, agg5#253, agg6#254, agg7#255] +Keys [1]: [i_item_id#29] +Functions [7]: [partial_avg(agg1#249), partial_avg(agg2#250), partial_avg(agg3#251), partial_avg(agg4#252), partial_avg(agg5#253), partial_avg(agg6#254), partial_avg(agg7#255)] +Aggregate Attributes [14]: [sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Results [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] + +(129) CometColumnarExchange +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(130) CometColumnarToRow [codegen id : 32] +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] + +(131) HashAggregate [codegen id : 32] +Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] +Keys [1]: [i_item_id#29] +Functions [7]: [avg(agg1#249), avg(agg2#250), avg(agg3#251), avg(agg4#252), avg(agg5#253), avg(agg6#254), avg(agg7#255)] +Aggregate Attributes [7]: [avg(agg1#249)#284, avg(agg2#250)#285, avg(agg3#251)#286, avg(agg4#252)#287, avg(agg5#253)#288, avg(agg6#254)#289, avg(agg7#255)#290] +Results [11]: [i_item_id#29, null AS ca_country#291, null AS ca_state#292, null AS county#293, avg(agg1#249)#284 AS agg1#294, avg(agg2#250)#285 AS agg2#295, avg(agg3#251)#286 AS agg3#296, avg(agg4#252)#287 AS agg4#297, avg(agg5#253)#288 AS agg5#298, avg(agg6#254)#289 AS agg6#299, avg(agg7#255)#290 AS agg7#300] + +(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] +PartitionFilters: [isnotnull(cs_sold_date_sk#309), dynamicpruningexpression(cs_sold_date_sk#309 IN dynamicpruning#310)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(120) CometFilter -Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) +(133) CometFilter +Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +Condition : ((isnotnull(cs_bill_cdemo_sk#302) AND isnotnull(cs_bill_customer_sk#301)) AND isnotnull(cs_item_sk#303)) -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#194, cd_dep_count#195] +(134) CometColumnarToRow [codegen id : 39] +Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Right output [2]: [cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight +(135) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#311, cd_dep_count#312] -(123) CometProject -Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] +(136) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_cdemo_sk#302] +Right keys [1]: [cd_demo_sk#311] +Join type: Inner +Join condition: None -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +(137) Project [codegen id : 39] +Output [9]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312] +Input [11]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_demo_sk#311, cd_dep_count#312] -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight +(138) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -(126) CometProject -Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +(139) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_customer_sk#301] +Right keys [1]: [c_customer_sk#313] +Join type: Inner +Join condition: None -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#200] +(140) Project [codegen id : 39] +Output [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +Input [13]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [cd_demo_sk#200] -Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight +(141) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#317] -(129) CometProject -Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] +(142) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_cdemo_sk#314] +Right keys [1]: [cd_demo_sk#317] +Join type: Inner +Join condition: None -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#201] +(143) Project [codegen id : 39] +Output [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] +Input [12]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [ca_address_sk#201] -Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight +(144) ReusedExchange [Reuses operator id: 119] +Output [1]: [ca_address_sk#318] -(132) CometProject -Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] +(145) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_addr_sk#315] +Right keys [1]: [ca_address_sk#318] +Join type: Inner +Join condition: None -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#202] +(146) Project [codegen id : 39] +Output [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316] +Input [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -Right output [1]: [d_date_sk#202] -Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight +(147) ReusedExchange [Reuses operator id: 168] +Output [1]: [d_date_sk#319] -(135) CometProject -Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] +(148) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_sold_date_sk#309] +Right keys [1]: [d_date_sk#319] +Join type: Inner +Join condition: None -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#203] +(149) Project [codegen id : 39] +Output [8]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316] +Input [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316, d_date_sk#319] + +(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#320] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(137) CometFilter -Input [1]: [i_item_sk#203] -Condition : isnotnull(i_item_sk#203) +(151) CometFilter +Input [1]: [i_item_sk#320] +Condition : isnotnull(i_item_sk#320) + +(152) CometColumnarToRow [codegen id : 38] +Input [1]: [i_item_sk#320] -(138) CometBroadcastExchange -Input [1]: [i_item_sk#203] -Arguments: [i_item_sk#203] +(153) BroadcastExchange +Input [1]: [i_item_sk#320] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -Right output [1]: [i_item_sk#203] -Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight +(154) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_item_sk#303] +Right keys [1]: [i_item_sk#320] +Join type: Inner +Join condition: None -(140) CometProject -Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] -Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] +(155) Project [codegen id : 39] +Output [7]: [cast(cs_quantity#304 as decimal(12,2)) AS agg1#321, cast(cs_list_price#305 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#307 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#306 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#308 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] +Input [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316, i_item_sk#320] -(141) CometHashAggregate -Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] +(156) HashAggregate [codegen id : 39] +Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] Keys: [] -Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] +Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] +Aggregate Attributes [14]: [sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339, sum#340, count#341] +Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -(142) CometExchange -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(157) CometColumnarExchange +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(143) CometHashAggregate -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] +(158) CometColumnarToRow [codegen id : 40] +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] + +(159) HashAggregate [codegen id : 40] +Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] Keys: [] -Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] +Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] +Aggregate Attributes [7]: [avg(agg1#321)#356, avg(agg2#322)#357, avg(agg3#323)#358, avg(agg4#324)#359, avg(agg5#325)#360, avg(agg6#326)#361, avg(agg7#327)#362] +Results [11]: [null AS i_item_id#363, null AS ca_country#364, null AS ca_state#365, null AS county#366, avg(agg1#321)#356 AS agg1#367, avg(agg2#322)#357 AS agg2#368, avg(agg3#323)#358 AS agg3#369, avg(agg4#324)#359 AS agg4#370, avg(agg5#325)#360 AS agg5#371, avg(agg6#326)#361 AS agg6#372, avg(agg7#327)#362 AS agg7#373] + +(160) Union -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] -Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] -Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] -Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] +(161) RowToColumnar +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +(162) CometTakeOrderedAndProject +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#31 ASC NULLS FIRST,ca_state#32 ASC NULLS FIRST,ca_county#33 ASC NULLS FIRST,i_item_id#30 ASC NULLS FIRST], output=[i_item_id#30,ca_country#31,ca_state#32,ca_county#33,agg1#76,agg2#77,agg3#78,agg4#79,agg5#80,agg6#81,agg7#82]), [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82], 100, 0, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +(163) CometColumnarToRow [codegen id : 41] +Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) +BroadcastExchange (168) ++- * CometColumnarToRow (167) + +- CometProject (166) + +- CometFilter (165) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (164) -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] +(164) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#374] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) +(165) CometFilter +Input [2]: [d_date_sk#26, d_year#374] +Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_date_sk#26)) -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] +(166) CometProject +Input [2]: [d_date_sk#26, d_year#374] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(167) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(151) BroadcastExchange +(168) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#163 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#236 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#309 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt index b18a444bb9..7a8f23c282 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt @@ -1,214 +1,255 @@ CometColumnarToRow +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 103 out of 210 eligible operators (49%). Final plan contains 43 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index 1572a2a240..91800fcca0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -1,160 +1,245 @@ -WholeStageCodegen (1) +WholeStageCodegen (41) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - CometBroadcastExchange [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] - CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - ReusedExchange [cd_demo_sk] #5 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (16) + HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country,ca_state] #8 + WholeStageCodegen (15) + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (24) + HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,ca_country] #10 + WholeStageCodegen (23) + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (32) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #12 + WholeStageCodegen (31) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (28) + CometColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (40) + HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (39) + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + ReusedExchange [ca_address_sk] #13 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (38) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt index 322bbe63b3..d8ae1972ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- RowToColumnar (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -36,129 +38,134 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(16) CometExchange +(15) CometColumnarExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometHashAggregate +(16) CometColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] + +(23) RowToColumnar +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(24) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(27) CometFilter +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(28) CometProject +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index 400633ed9f..a2cde6785b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -1,31 +1,37 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt index cf18e68a3d..5a0729e0da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt @@ -1,35 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + RowToColumnar + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_class] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt index 322bbe63b3..d8ae1972ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt @@ -1,27 +1,29 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- RowToColumnar (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -36,129 +38,134 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(16) CometExchange +(15) CometColumnarExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometHashAggregate +(16) CometColumnarToRow [codegen id : 4] +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] + +(23) RowToColumnar +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(24) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(27) CometFilter +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(28) CometProject +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(28) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt index 400633ed9f..a2cde6785b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt @@ -1,31 +1,37 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index cf18e68a3d..5a0729e0da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -1,35 +1,47 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + RowToColumnar + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_class] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt index f85c898208..295d35ec0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt @@ -1,30 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) +* CometColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- RowToColumnar (24) + +- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometColumnarExchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastNestedLoopJoin Inner BuildRight (17) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (11) + : +- * CometColumnarToRow (10) + : +- CometProject (9) + : +- CometFilter (8) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + +- BroadcastExchange (16) + +- * CometColumnarToRow (15) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -39,114 +39,109 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) +(3) CometColumnarToRow [codegen id : 4] +Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 31] +Output [1]: [d_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) +(8) CometFilter +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Condition : isnotnull(i_item_sk#6) -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] +(9) CometProject +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(10) CometColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight +(11) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(13) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 3] Input: [] -(18) BroadcastExchange +(16) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] +Arguments: IdentityBroadcastMode, [plan_id=2] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(17) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] +Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(19) Expand [codegen id : 4] +Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] +Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(20) HashAggregate [codegen id : 4] +Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Aggregate Attributes [2]: [sum#20, count#21] +Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +(22) CometColumnarToRow [codegen id : 5] +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(23) HashAggregate [codegen id : 5] +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] +Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] + +(24) RowToColumnar +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] + +(25) CometTakeOrderedAndProject +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#15 ASC NULLS FIRST,i_brand#16 ASC NULLS FIRST,i_class#17 ASC NULLS FIRST,i_category#18 ASC NULLS FIRST], output=[i_product_name#15,i_brand#16,i_class#17,i_category#18,qoh#25]), [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] +(26) CometColumnarToRow [codegen id : 6] +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] ===== Subqueries ===== @@ -159,18 +154,18 @@ BroadcastExchange (31) (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +Output [2]: [d_date_sk#5, d_month_seq#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#26] +Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) (29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +Input [2]: [d_date_sk#5, d_month_seq#26] Arguments: [d_date_sk#5], [d_date_sk#5] (30) CometColumnarToRow [codegen id : 1] @@ -178,6 +173,6 @@ Input [1]: [d_date_sk#5] (31) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index 261903252b..d5e580d4a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ - TakeOrderedAndProject [COMET: ] -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt index 221c6063ce..00be4f0608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt @@ -1,41 +1,47 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt index f85c898208..295d35ec0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt @@ -1,30 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) +* CometColumnarToRow (26) ++- CometTakeOrderedAndProject (25) + +- RowToColumnar (24) + +- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometColumnarExchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastNestedLoopJoin Inner BuildRight (17) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (11) + : +- * CometColumnarToRow (10) + : +- CometProject (9) + : +- CometFilter (8) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + +- BroadcastExchange (16) + +- * CometColumnarToRow (15) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -39,114 +39,109 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) +(3) CometColumnarToRow [codegen id : 4] +Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 31] +Output [1]: [d_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) +(8) CometFilter +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Condition : isnotnull(i_item_sk#6) -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] +(9) CometProject +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(10) CometColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight +(11) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(13) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 3] Input: [] -(18) BroadcastExchange +(16) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] +Arguments: IdentityBroadcastMode, [plan_id=2] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(17) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] +Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(19) Expand [codegen id : 4] +Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] +Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(20) HashAggregate [codegen id : 4] +Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Aggregate Attributes [2]: [sum#20, count#21] +Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometColumnarExchange +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +(22) CometColumnarToRow [codegen id : 5] +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +(23) HashAggregate [codegen id : 5] +Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] +Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] + +(24) RowToColumnar +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] + +(25) CometTakeOrderedAndProject +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#15 ASC NULLS FIRST,i_brand#16 ASC NULLS FIRST,i_class#17 ASC NULLS FIRST,i_category#18 ASC NULLS FIRST], output=[i_product_name#15,i_brand#16,i_class#17,i_category#18,qoh#25]), [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] +(26) CometColumnarToRow [codegen id : 6] +Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] ===== Subqueries ===== @@ -159,18 +154,18 @@ BroadcastExchange (31) (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +Output [2]: [d_date_sk#5, d_month_seq#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#26] +Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) (29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +Input [2]: [d_date_sk#5, d_month_seq#26] Arguments: [d_date_sk#5], [d_date_sk#5] (30) CometColumnarToRow [codegen id : 1] @@ -178,6 +173,6 @@ Input [1]: [d_date_sk#5] (31) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt index 261903252b..d5e580d4a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt @@ -1,34 +1,38 @@ - TakeOrderedAndProject [COMET: ] -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 221c6063ce..00be4f0608 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -1,41 +1,47 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] + RowToColumnar + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt index bf16cc4d21..34bb4d3071 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt @@ -1,51 +1,61 @@ == Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) +* CometColumnarToRow (57) ++- CometTakeOrderedAndProject (56) + +- RowToColumnar (55) + +- Union (54) + :- * HashAggregate (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * CometColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) + :- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- CometColumnarExchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- * CometColumnarToRow (27) + : +- ReusedExchange (26) + :- * HashAggregate (39) + : +- * CometColumnarToRow (38) + : +- CometColumnarExchange (37) + : +- * HashAggregate (36) + : +- * HashAggregate (35) + : +- * CometColumnarToRow (34) + : +- ReusedExchange (33) + :- * HashAggregate (46) + : +- * CometColumnarToRow (45) + : +- CometColumnarExchange (44) + : +- * HashAggregate (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- ReusedExchange (40) + +- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- ReusedExchange (47) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -60,242 +70,295 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] +(3) CometColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] +(4) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#6] -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) +(8) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) + +(9) CometProject +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] +(10) CometColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(11) BroadcastExchange +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(13) Project [codegen id : 4] +Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight +(15) CometFilter +Input [1]: [w_warehouse_sk#16] +Condition : isnotnull(w_warehouse_sk#16) -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(16) CometColumnarToRow [codegen id : 3] +Input [1]: [w_warehouse_sk#16] -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] +(17) BroadcastExchange +Input [1]: [w_warehouse_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#16] +Join type: Inner +Join condition: None -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] +(19) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(20) HashAggregate [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [sum#17, count#18] +Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#30)] +(21) CometColumnarExchange +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] +(22) CometColumnarToRow [codegen id : 5] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +(23) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] + +(24) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] +Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] +Functions [1]: [partial_avg(qoh#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] + +(25) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] +Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] +Functions [1]: [avg(qoh#26)] +Aggregate Attributes [1]: [avg(qoh#26)#31] +Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] + +(26) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] + +(27) CometColumnarToRow [codegen id : 10] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] + +(28) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] +Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#36] -Keys [2]: [i_product_name#16, i_brand#13] +(29) HashAggregate [codegen id : 10] +Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] +Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#36)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] +(30) CometColumnarExchange +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#41)] +(31) CometColumnarToRow [codegen id : 11] +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#42] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#42)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#43, count#44] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#43, count#44] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#42)] +(32) HashAggregate [codegen id : 11] +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] + +(33) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] + +(34) CometColumnarToRow [codegen id : 16] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] + +(35) HashAggregate [codegen id : 16] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#46)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] +Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] + +(36) HashAggregate [codegen id : 16] +Input [3]: [i_product_name#15, i_brand#12, qoh#47] +Keys [2]: [i_product_name#15, i_brand#12] +Functions [1]: [partial_avg(qoh#47)] +Aggregate Attributes [2]: [sum#48, count#49] +Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] + +(37) CometColumnarExchange +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(38) CometColumnarToRow [codegen id : 17] +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] + +(39) HashAggregate [codegen id : 17] +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] +Keys [2]: [i_product_name#15, i_brand#12] +Functions [1]: [avg(qoh#47)] +Aggregate Attributes [1]: [avg(qoh#47)#52] +Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] (40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#47)] - -(42) CometHashAggregate -Input [1]: [qoh#48] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] + +(41) CometColumnarToRow [codegen id : 22] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] + +(42) HashAggregate [codegen id : 22] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#58)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] +Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] + +(43) HashAggregate [codegen id : 22] +Input [2]: [i_product_name#15, qoh#59] +Keys [1]: [i_product_name#15] +Functions [1]: [partial_avg(qoh#59)] +Aggregate Attributes [2]: [sum#60, count#61] +Results [3]: [i_product_name#15, sum#62, count#63] + +(44) CometColumnarExchange +Input [3]: [i_product_name#15, sum#62, count#63] +Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(45) CometColumnarToRow [codegen id : 23] +Input [3]: [i_product_name#15, sum#62, count#63] + +(46) HashAggregate [codegen id : 23] +Input [3]: [i_product_name#15, sum#62, count#63] +Keys [1]: [i_product_name#15] +Functions [1]: [avg(qoh#59)] +Aggregate Attributes [1]: [avg(qoh#59)#64] +Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS i_category#67, avg(qoh#59)#64 AS qoh#68] + +(47) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] + +(48) CometColumnarToRow [codegen id : 28] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] + +(49) HashAggregate [codegen id : 28] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#71)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] +Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] + +(50) HashAggregate [codegen id : 28] +Input [1]: [qoh#72] Keys: [] -Functions [1]: [partial_avg(qoh#48)] +Functions [1]: [partial_avg(qoh#72)] +Aggregate Attributes [2]: [sum#73, count#74] +Results [2]: [sum#75, count#76] + +(51) CometColumnarExchange +Input [2]: [sum#75, count#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(43) CometExchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(52) CometColumnarToRow [codegen id : 29] +Input [2]: [sum#75, count#76] -(44) CometHashAggregate -Input [2]: [sum#49, count#50] +(53) HashAggregate [codegen id : 29] +Input [2]: [sum#75, count#76] Keys: [] -Functions [1]: [avg(qoh#48)] +Functions [1]: [avg(qoh#72)] +Aggregate Attributes [1]: [avg(qoh#72)#77] +Results [5]: [null AS i_product_name#78, null AS i_brand#79, null AS i_class#80, null AS i_category#81, avg(qoh#72)#77 AS qoh#82] + +(54) Union -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] -Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] -Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] +(55) RowToColumnar +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +(56) CometTakeOrderedAndProject +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#32 ASC NULLS FIRST,i_product_name#22 ASC NULLS FIRST,i_brand#23 ASC NULLS FIRST,i_class#24 ASC NULLS FIRST,i_category#25 ASC NULLS FIRST], output=[i_product_name#22,i_brand#23,i_class#24,i_category#25,qoh#32]), [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32], 100, 0, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +(57) CometColumnarToRow [codegen id : 30] +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#83] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) +(59) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#83] +Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_month_seq#83 <= 1223)) AND isnotnull(d_date_sk#6)) -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] +(60) CometProject +Input [2]: [d_date_sk#6, d_month_seq#83] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(52) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index c018673888..08f1fe4f48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -1,159 +1,189 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 75 out of 151 eligible operators (49%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt index b59605103e..d855717f2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt @@ -1,57 +1,94 @@ -WholeStageCodegen (1) +WholeStageCodegen (30) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (11) + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class] #5 + WholeStageCodegen (10) + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (17) + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand] #6 + WholeStageCodegen (16) + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (23) + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name] #7 + WholeStageCodegen (22) + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (29) + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (28) + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt index bf16cc4d21..34bb4d3071 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt @@ -1,51 +1,61 @@ == Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) +* CometColumnarToRow (57) ++- CometTakeOrderedAndProject (56) + +- RowToColumnar (55) + +- Union (54) + :- * HashAggregate (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * CometColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) + :- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- CometColumnarExchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- * CometColumnarToRow (27) + : +- ReusedExchange (26) + :- * HashAggregate (39) + : +- * CometColumnarToRow (38) + : +- CometColumnarExchange (37) + : +- * HashAggregate (36) + : +- * HashAggregate (35) + : +- * CometColumnarToRow (34) + : +- ReusedExchange (33) + :- * HashAggregate (46) + : +- * CometColumnarToRow (45) + : +- CometColumnarExchange (44) + : +- * HashAggregate (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- ReusedExchange (40) + +- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- ReusedExchange (47) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -60,242 +70,295 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] +(3) CometColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] +(4) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#6] -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) +(8) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) + +(9) CometProject +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] +(10) CometColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(11) BroadcastExchange +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(13) Project [codegen id : 4] +Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight +(15) CometFilter +Input [1]: [w_warehouse_sk#16] +Condition : isnotnull(w_warehouse_sk#16) -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +(16) CometColumnarToRow [codegen id : 3] +Input [1]: [w_warehouse_sk#16] -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] +(17) BroadcastExchange +Input [1]: [w_warehouse_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#16] +Join type: Inner +Join condition: None -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] +(19) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(20) HashAggregate [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [sum#17, count#18] +Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#30)] +(21) CometColumnarExchange +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] +(22) CometColumnarToRow [codegen id : 5] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +(23) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] + +(24) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] +Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] +Functions [1]: [partial_avg(qoh#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] + +(25) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] +Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] +Functions [1]: [avg(qoh#26)] +Aggregate Attributes [1]: [avg(qoh#26)#31] +Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] + +(26) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] + +(27) CometColumnarToRow [codegen id : 10] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] + +(28) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] +Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#36] -Keys [2]: [i_product_name#16, i_brand#13] +(29) HashAggregate [codegen id : 10] +Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] +Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#36)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] +(30) CometColumnarExchange +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#41)] +(31) CometColumnarToRow [codegen id : 11] +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#42] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#42)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#43, count#44] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#43, count#44] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#42)] +(32) HashAggregate [codegen id : 11] +Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] + +(33) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] + +(34) CometColumnarToRow [codegen id : 16] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] + +(35) HashAggregate [codegen id : 16] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#46)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] +Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] + +(36) HashAggregate [codegen id : 16] +Input [3]: [i_product_name#15, i_brand#12, qoh#47] +Keys [2]: [i_product_name#15, i_brand#12] +Functions [1]: [partial_avg(qoh#47)] +Aggregate Attributes [2]: [sum#48, count#49] +Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] + +(37) CometColumnarExchange +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] +Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(38) CometColumnarToRow [codegen id : 17] +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] + +(39) HashAggregate [codegen id : 17] +Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] +Keys [2]: [i_product_name#15, i_brand#12] +Functions [1]: [avg(qoh#47)] +Aggregate Attributes [1]: [avg(qoh#47)#52] +Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] (40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#47)] - -(42) CometHashAggregate -Input [1]: [qoh#48] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] + +(41) CometColumnarToRow [codegen id : 22] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] + +(42) HashAggregate [codegen id : 22] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#58)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] +Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] + +(43) HashAggregate [codegen id : 22] +Input [2]: [i_product_name#15, qoh#59] +Keys [1]: [i_product_name#15] +Functions [1]: [partial_avg(qoh#59)] +Aggregate Attributes [2]: [sum#60, count#61] +Results [3]: [i_product_name#15, sum#62, count#63] + +(44) CometColumnarExchange +Input [3]: [i_product_name#15, sum#62, count#63] +Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(45) CometColumnarToRow [codegen id : 23] +Input [3]: [i_product_name#15, sum#62, count#63] + +(46) HashAggregate [codegen id : 23] +Input [3]: [i_product_name#15, sum#62, count#63] +Keys [1]: [i_product_name#15] +Functions [1]: [avg(qoh#59)] +Aggregate Attributes [1]: [avg(qoh#59)#64] +Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS i_category#67, avg(qoh#59)#64 AS qoh#68] + +(47) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] + +(48) CometColumnarToRow [codegen id : 28] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] + +(49) HashAggregate [codegen id : 28] +Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] +Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +Functions [1]: [avg(inv_quantity_on_hand#71)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] +Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] + +(50) HashAggregate [codegen id : 28] +Input [1]: [qoh#72] Keys: [] -Functions [1]: [partial_avg(qoh#48)] +Functions [1]: [partial_avg(qoh#72)] +Aggregate Attributes [2]: [sum#73, count#74] +Results [2]: [sum#75, count#76] + +(51) CometColumnarExchange +Input [2]: [sum#75, count#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(43) CometExchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(52) CometColumnarToRow [codegen id : 29] +Input [2]: [sum#75, count#76] -(44) CometHashAggregate -Input [2]: [sum#49, count#50] +(53) HashAggregate [codegen id : 29] +Input [2]: [sum#75, count#76] Keys: [] -Functions [1]: [avg(qoh#48)] +Functions [1]: [avg(qoh#72)] +Aggregate Attributes [1]: [avg(qoh#72)#77] +Results [5]: [null AS i_product_name#78, null AS i_brand#79, null AS i_class#80, null AS i_category#81, avg(qoh#72)#77 AS qoh#82] + +(54) Union -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] -Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] -Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] +(55) RowToColumnar +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +(56) CometTakeOrderedAndProject +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#32 ASC NULLS FIRST,i_product_name#22 ASC NULLS FIRST,i_brand#23 ASC NULLS FIRST,i_class#24 ASC NULLS FIRST,i_category#25 ASC NULLS FIRST], output=[i_product_name#22,i_brand#23,i_class#24,i_category#25,qoh#32]), [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32], 100, 0, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +(57) CometColumnarToRow [codegen id : 30] +Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#83] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) +(59) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#83] +Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_month_seq#83 <= 1223)) AND isnotnull(d_date_sk#6)) -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] +(60) CometProject +Input [2]: [d_date_sk#6, d_month_seq#83] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(52) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt index c018673888..08f1fe4f48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt @@ -1,159 +1,189 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 75 out of 151 eligible operators (49%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index b59605103e..d855717f2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -1,57 +1,94 @@ -WholeStageCodegen (1) +WholeStageCodegen (30) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] - CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (11) + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class] #5 + WholeStageCodegen (10) + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (17) + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand] #6 + WholeStageCodegen (16) + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (23) + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name] #7 + WholeStageCodegen (22) + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (29) + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (28) + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt index 17ef709df5..fa411d777f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt @@ -1,54 +1,57 @@ == Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) +* CometColumnarToRow (53) ++- CometSort (52) + +- CometColumnarExchange (51) + +- * Filter (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- CometColumnarExchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * CometColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,344 +109,363 @@ Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7 Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) -(15) CometProject +(16) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] -(16) CometBroadcastExchange +(17) CometColumnarToRow [codegen id : 1] +Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] + +(18) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None -(18) CometProject +(20) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter +(22) CometFilter Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) -(21) CometProject +(23) CometProject Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] -(22) CometBroadcastExchange +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] + +(25) BroadcastExchange Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight +(26) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None -(24) CometProject +(27) Project [codegen id : 5] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(26) CometFilter +(29) CometFilter Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) -(27) CometProject +(30) CometProject Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] -(28) CometBroadcastExchange +(31) CometColumnarToRow [codegen id : 3] Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight +(32) BroadcastExchange +Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#26] +Join type: Inner +Join condition: None -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(34) Project [codegen id : 5] +Output [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] ReadSchema: struct -(33) CometFilter +(36) CometFilter Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) -(34) CometProject +(37) CometProject Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 4] Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(36) BroadcastExchange +(39) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 2] +(40) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(41) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(42) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#39] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(40) CometColumnarExchange +(43) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(41) CometColumnarToRow [codegen id : 3] +(44) CometColumnarToRow [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(45) HashAggregate [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(46) HashAggregate [codegen id : 6] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(44) CometColumnarExchange +(47) CometColumnarExchange Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(45) CometColumnarToRow [codegen id : 4] +(48) CometColumnarToRow [codegen id : 7] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(49) HashAggregate [codegen id : 7] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(50) Filter [codegen id : 7] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) -(48) CometColumnarExchange +(51) CometColumnarExchange Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(49) CometSort +(52) CometSort Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(53) CometColumnarToRow [codegen id : 8] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 50 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (84) ++- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * Project (76) + +- * BroadcastHashJoin Inner BuildRight (75) + :- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * CometColumnarToRow (60) + : : : : +- CometProject (59) + : : : : +- CometSortMergeJoin (58) + : : : : :- CometSort (55) + : : : : : +- ReusedExchange (54) + : : : : +- CometSort (57) + : : : : +- ReusedExchange (56) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (68) + : : +- * CometColumnarToRow (67) + : : +- CometProject (66) + : : +- CometFilter (65) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (64) + : +- ReusedExchange (71) + +- ReusedExchange (74) + + +(54) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -(52) CometSort +(55) CometSort Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] -(53) ReusedExchange [Reuses operator id: 9] +(56) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#56, sr_ticket_number#57] -(54) CometSort +(57) CometSort Input [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] -(55) CometSortMergeJoin +(58) CometSortMergeJoin Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Right output [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner -(56) CometProject +(59) CometProject Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -(57) ReusedExchange [Reuses operator id: 16] +(60) CometColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] + +(61) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#53] +Right keys [1]: [s_store_sk#58] +Join type: Inner +Join condition: None -(59) CometProject +(63) Project [codegen id : 5] +Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter +(65) CometFilter Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Condition : isnotnull(i_item_sk#60) -(62) CometProject +(66) CometProject Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] -(63) CometBroadcastExchange +(67) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] + +(68) BroadcastExchange Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight +(69) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#51] +Right keys [1]: [i_item_sk#60] +Join type: Inner +Join condition: None -(65) CometProject +(70) Project [codegen id : 5] +Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(66) ReusedExchange [Reuses operator id: 28] +(71) ReusedExchange [Reuses operator id: 32] Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight +(72) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_customer_sk#52] +Right keys [1]: [c_customer_sk#66] +Join type: Inner +Join condition: None -(68) CometProject +(73) Project [codegen id : 5] +Output [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(70) ReusedExchange [Reuses operator id: 36] +(74) ReusedExchange [Reuses operator id: 39] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(75) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(76) Project [codegen id : 5] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(77) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum#71] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(74) CometColumnarExchange +(78) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(75) CometColumnarToRow [codegen id : 3] +(79) CometColumnarToRow [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(80) HashAggregate [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(77) HashAggregate [codegen id : 3] +(81) HashAggregate [codegen id : 6] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] Aggregate Attributes [2]: [sum#74, count#75] Results [2]: [sum#76, count#77] -(78) CometColumnarExchange +(82) CometColumnarExchange Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(79) CometColumnarToRow [codegen id : 4] +(83) CometColumnarToRow [codegen id : 7] Input [2]: [sum#76, count#77] -(80) HashAggregate [codegen id : 4] +(84) HashAggregate [codegen id : 7] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt index 7847aefd83..0ed56d0b0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt @@ -13,34 +13,37 @@ CometColumnarToRow : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- 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 + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,34 +62,37 @@ CometColumnarToRow +- HashAggregate +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- 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 + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -96,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 88 eligible operators (61%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt index 62f492f632..ce751cb78d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt @@ -1,70 +1,76 @@ -WholeStageCodegen (5) +WholeStageCodegen (8) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (7) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (7) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (6) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (6) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] @@ -77,21 +83,33 @@ WholeStageCodegen (5) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) + WholeStageCodegen (4) CometColumnarToRow InputAdapter CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt index 17ef709df5..fa411d777f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt @@ -1,54 +1,57 @@ == Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) +* CometColumnarToRow (53) ++- CometSort (52) + +- CometColumnarExchange (51) + +- * Filter (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- CometColumnarExchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * CometColumnarToRow (13) + : : : : +- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) + : : : +- BroadcastExchange (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,344 +109,363 @@ Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7 Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(15) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) -(15) CometProject +(16) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] -(16) CometBroadcastExchange +(17) CometColumnarToRow [codegen id : 1] +Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] + +(18) BroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None -(18) CometProject +(20) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter +(22) CometFilter Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) -(21) CometProject +(23) CometProject Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] -(22) CometBroadcastExchange +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] + +(25) BroadcastExchange Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight +(26) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None -(24) CometProject +(27) Project [codegen id : 5] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(26) CometFilter +(29) CometFilter Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) -(27) CometProject +(30) CometProject Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] -(28) CometBroadcastExchange +(31) CometColumnarToRow [codegen id : 3] Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight +(32) BroadcastExchange +Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#26] +Join type: Inner +Join condition: None -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(34) Project [codegen id : 5] +Output [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] ReadSchema: struct -(33) CometFilter +(36) CometFilter Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) -(34) CometProject +(37) CometProject Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 4] Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(36) BroadcastExchange +(39) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 2] +(40) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(41) Project [codegen id : 5] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(42) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#39] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(40) CometColumnarExchange +(43) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(41) CometColumnarToRow [codegen id : 3] +(44) CometColumnarToRow [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(45) HashAggregate [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(46) HashAggregate [codegen id : 6] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(44) CometColumnarExchange +(47) CometColumnarExchange Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(45) CometColumnarToRow [codegen id : 4] +(48) CometColumnarToRow [codegen id : 7] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(49) HashAggregate [codegen id : 7] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(50) Filter [codegen id : 7] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) -(48) CometColumnarExchange +(51) CometColumnarExchange Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(49) CometSort +(52) CometSort Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(53) CometColumnarToRow [codegen id : 8] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 50 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (84) ++- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * Project (76) + +- * BroadcastHashJoin Inner BuildRight (75) + :- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (63) + : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : :- * CometColumnarToRow (60) + : : : : +- CometProject (59) + : : : : +- CometSortMergeJoin (58) + : : : : :- CometSort (55) + : : : : : +- ReusedExchange (54) + : : : : +- CometSort (57) + : : : : +- ReusedExchange (56) + : : : +- ReusedExchange (61) + : : +- BroadcastExchange (68) + : : +- * CometColumnarToRow (67) + : : +- CometProject (66) + : : +- CometFilter (65) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (64) + : +- ReusedExchange (71) + +- ReusedExchange (74) + + +(54) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -(52) CometSort +(55) CometSort Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] -(53) ReusedExchange [Reuses operator id: 9] +(56) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#56, sr_ticket_number#57] -(54) CometSort +(57) CometSort Input [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] -(55) CometSortMergeJoin +(58) CometSortMergeJoin Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Right output [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner -(56) CometProject +(59) CometProject Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -(57) ReusedExchange [Reuses operator id: 16] +(60) CometColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] + +(61) ReusedExchange [Reuses operator id: 18] Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#53] +Right keys [1]: [s_store_sk#58] +Join type: Inner +Join condition: None -(59) CometProject +(63) Project [codegen id : 5] +Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(61) CometFilter +(65) CometFilter Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Condition : isnotnull(i_item_sk#60) -(62) CometProject +(66) CometProject Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] -(63) CometBroadcastExchange +(67) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] + +(68) BroadcastExchange Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight +(69) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#51] +Right keys [1]: [i_item_sk#60] +Join type: Inner +Join condition: None -(65) CometProject +(70) Project [codegen id : 5] +Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(66) ReusedExchange [Reuses operator id: 28] +(71) ReusedExchange [Reuses operator id: 32] Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight +(72) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_customer_sk#52] +Right keys [1]: [c_customer_sk#66] +Join type: Inner +Join condition: None -(68) CometProject +(73) Project [codegen id : 5] +Output [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(70) ReusedExchange [Reuses operator id: 36] +(74) ReusedExchange [Reuses operator id: 39] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(75) BroadcastHashJoin [codegen id : 5] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(76) Project [codegen id : 5] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(77) HashAggregate [codegen id : 5] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum#71] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(74) CometColumnarExchange +(78) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(75) CometColumnarToRow [codegen id : 3] +(79) CometColumnarToRow [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(80) HashAggregate [codegen id : 6] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(77) HashAggregate [codegen id : 3] +(81) HashAggregate [codegen id : 6] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] Aggregate Attributes [2]: [sum#74, count#75] Results [2]: [sum#76, count#77] -(78) CometColumnarExchange +(82) CometColumnarExchange Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(79) CometColumnarToRow [codegen id : 4] +(83) CometColumnarToRow [codegen id : 7] Input [2]: [sum#76, count#77] -(80) HashAggregate [codegen id : 4] +(84) HashAggregate [codegen id : 7] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt index 7847aefd83..0ed56d0b0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt @@ -13,34 +13,37 @@ CometColumnarToRow : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- 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 + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,34 +62,37 @@ CometColumnarToRow +- HashAggregate +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- 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 + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -96,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 88 eligible operators (61%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 62f492f632..ce751cb78d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -1,70 +1,76 @@ -WholeStageCodegen (5) +WholeStageCodegen (8) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (7) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (7) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (6) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (6) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] @@ -77,21 +83,33 @@ WholeStageCodegen (5) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) + WholeStageCodegen (4) CometColumnarToRow InputAdapter CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt index 397a3eb10c..876b7c356b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt @@ -1,75 +1,84 @@ == Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) +* CometColumnarToRow (80) ++- CometTakeOrderedAndProject (79) + +- RowToColumnar (78) + +- Union (77) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (18) + : : +- * CometColumnarToRow (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (25) + : +- * CometColumnarToRow (24) + : +- CometProject (23) + : +- CometFilter (22) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + :- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * Project (40) + : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : :- * Project (37) + : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * CometColumnarToRow (34) + : : : : : +- CometFilter (33) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometProject (43) + : : +- CometFilter (42) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (41) + : +- ReusedExchange (48) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * CometColumnarToRow (57) + : : : : +- CometFilter (56) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (55) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- ReusedExchange (64) + +- BroadcastExchange (70) + +- * CometColumnarToRow (69) + +- CometFilter (68) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (67) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -84,354 +93,395 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) -(5) CometProject +(6) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [1]: [cd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) +(15) CometFilter +Input [2]: [s_store_sk#15, s_state#16] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) + +(16) CometProject +Input [2]: [s_store_sk#15, s_state#16] +Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] +(17) CometColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#15, s_state#17] -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] +(18) BroadcastExchange +Input [2]: [s_store_sk#15, s_state#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] +(20) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] (21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) (23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] + +(24) CometColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#18, i_item_id#20] + +(25) BroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(26) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 5] +Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] + +(28) HashAggregate [codegen id : 5] +Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] +Keys [2]: [i_item_id#21, s_state#22] +Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] +Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] +Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(29) CometColumnarExchange +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 6] +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(31) HashAggregate [codegen id : 6] +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Keys [2]: [i_item_id#21, s_state#22] +Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] +Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] +Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) +(33) CometFilter +Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) + +(34) CometColumnarToRow [codegen id : 11] +Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] +(35) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#61] -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_cdemo_sk#53] +Right keys [1]: [cd_demo_sk#61] +Join type: Inner +Join condition: None -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +(37) Project [codegen id : 11] +Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#61] -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] +(38) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#62] -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#62] +Join type: Inner +Join condition: None -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +(40) Project [codegen id : 11] +Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] +Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#62] -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight +(42) CometFilter +Input [2]: [s_store_sk#63, s_state#64] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#64, 2, true, false, true) = TN) AND isnotnull(s_store_sk#63)) (43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Input [2]: [s_store_sk#63, s_state#64] +Arguments: [s_store_sk#63], [s_store_sk#63] + +(44) CometColumnarToRow [codegen id : 9] +Input [1]: [s_store_sk#63] + +(45) BroadcastExchange +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#63] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 11] +Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] +Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#63] + +(48) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#65, i_item_id#20] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#65] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] +Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#20] + +(51) HashAggregate [codegen id : 11] +Input [5]: [i_item_id#20, agg1#66, agg2#67, agg3#68, agg4#69] +Keys [1]: [i_item_id#20] +Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] +Aggregate Attributes [8]: [sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] +Results [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] + +(52) CometColumnarExchange +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(53) CometColumnarToRow [codegen id : 12] +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] + +(54) HashAggregate [codegen id : 12] +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] +Keys [1]: [i_item_id#20] +Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] +Aggregate Attributes [4]: [avg(agg1#66)#86, avg(UnscaledValue(agg2#67))#87, avg(UnscaledValue(agg3#68))#88, avg(UnscaledValue(agg4#69))#89] +Results [7]: [i_item_id#20, null AS s_state#90, 1 AS g_state#91, avg(agg1#66)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#67))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#68))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#69))#89 / 100.0) as decimal(11,6)) AS agg4#95] + +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter -Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) +(56) CometFilter +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) + +(57) CometColumnarToRow [codegen id : 17] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#71] +(58) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#105] -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [cd_demo_sk#71] -Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight +(59) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#105] +Join type: Inner +Join condition: None -(54) CometProject -Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +(60) Project [codegen id : 17] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#105] -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#72] +(61) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#106] -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [d_date_sk#72] -Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#106] +Join type: Inner +Join condition: None -(57) CometProject -Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +(63) Project [codegen id : 17] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#73] +(64) ReusedExchange [Reuses operator id: 45] +Output [1]: [s_store_sk#107] -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [s_store_sk#73] -Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight +(65) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#107] +Join type: Inner +Join condition: None -(60) CometProject -Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] -Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +(66) Project [codegen id : 17] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#107] -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#74] +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(62) CometFilter -Input [1]: [i_item_sk#74] -Condition : isnotnull(i_item_sk#74) +(68) CometFilter +Input [1]: [i_item_sk#108] +Condition : isnotnull(i_item_sk#108) -(63) CometBroadcastExchange -Input [1]: [i_item_sk#74] -Arguments: [i_item_sk#74] +(69) CometColumnarToRow [codegen id : 16] +Input [1]: [i_item_sk#108] -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [i_item_sk#74] -Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight +(70) BroadcastExchange +Input [1]: [i_item_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(65) CometProject -Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] -Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] +(71) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#108] +Join type: Inner +Join condition: None -(66) CometHashAggregate -Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] +(72) Project [codegen id : 17] +Output [4]: [ss_quantity#99 AS agg1#109, ss_list_price#100 AS agg2#110, ss_coupon_amt#102 AS agg3#111, ss_sales_price#101 AS agg4#112] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#108] + +(73) HashAggregate [codegen id : 17] +Input [4]: [agg1#109, agg2#110, agg3#111, agg4#112] Keys: [] -Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] +Functions [4]: [partial_avg(agg1#109), partial_avg(UnscaledValue(agg2#110)), partial_avg(UnscaledValue(agg3#111)), partial_avg(UnscaledValue(agg4#112))] +Aggregate Attributes [8]: [sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] + +(74) CometColumnarExchange +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(67) CometExchange -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(75) CometColumnarToRow [codegen id : 18] +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] -(68) CometHashAggregate -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] +(76) HashAggregate [codegen id : 18] +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] Keys: [] -Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] +Functions [4]: [avg(agg1#109), avg(UnscaledValue(agg2#110)), avg(UnscaledValue(agg3#111)), avg(UnscaledValue(agg4#112))] +Aggregate Attributes [4]: [avg(agg1#109)#129, avg(UnscaledValue(agg2#110))#130, avg(UnscaledValue(agg3#111))#131, avg(UnscaledValue(agg4#112))#132] +Results [7]: [null AS i_item_id#133, null AS s_state#134, 1 AS g_state#135, avg(agg1#109)#129 AS agg1#136, cast((avg(UnscaledValue(agg2#110))#130 / 100.0) as decimal(11,6)) AS agg2#137, cast((avg(UnscaledValue(agg3#111))#131 / 100.0) as decimal(11,6)) AS agg3#138, cast((avg(UnscaledValue(agg4#112))#132 / 100.0) as decimal(11,6)) AS agg4#139] + +(77) Union -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] -Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] +(78) RowToColumnar +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +(79) CometTakeOrderedAndProject +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#21 ASC NULLS FIRST,s_state#22 ASC NULLS FIRST], output=[i_item_id#21,s_state#22,g_state#47,agg1#48,agg2#49,agg3#50,agg4#51]), [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51], 100, 0, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +(80) CometColumnarToRow [codegen id : 19] +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#140] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) +(82) CometFilter +Input [2]: [d_date_sk#14, d_year#140] +Condition : ((isnotnull(d_year#140) AND (d_year#140 = 1998)) AND isnotnull(d_date_sk#14)) -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] +(83) CometProject +Input [2]: [d_date_sk#14, d_year#140] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(85) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index affbc9ef37..c26488e3c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -1,99 +1,118 @@ CometColumnarToRow +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 48 out of 95 eligible operators (50%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt index 0d18ca5626..293a5fc379 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt @@ -1,83 +1,126 @@ -WholeStageCodegen (1) +WholeStageCodegen (19) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,s_state] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (17) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [s_store_sk] #7 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt index 397a3eb10c..876b7c356b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt @@ -1,75 +1,84 @@ == Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) +* CometColumnarToRow (80) ++- CometTakeOrderedAndProject (79) + +- RowToColumnar (78) + +- Union (77) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * Project (27) + : +- * BroadcastHashJoin Inner BuildRight (26) + : :- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (18) + : : +- * CometColumnarToRow (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (25) + : +- * CometColumnarToRow (24) + : +- CometProject (23) + : +- CometFilter (22) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + :- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * Project (40) + : : : +- * BroadcastHashJoin Inner BuildRight (39) + : : : :- * Project (37) + : : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : : :- * CometColumnarToRow (34) + : : : : : +- CometFilter (33) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometProject (43) + : : +- CometFilter (42) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (41) + : +- ReusedExchange (48) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (60) + : : : +- * BroadcastHashJoin Inner BuildRight (59) + : : : :- * CometColumnarToRow (57) + : : : : +- CometFilter (56) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (55) + : : : +- ReusedExchange (58) + : : +- ReusedExchange (61) + : +- ReusedExchange (64) + +- BroadcastExchange (70) + +- * CometColumnarToRow (69) + +- CometFilter (68) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (67) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -84,354 +93,395 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) -(5) CometProject +(6) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [1]: [cd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) +(15) CometFilter +Input [2]: [s_store_sk#15, s_state#16] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) + +(16) CometProject +Input [2]: [s_store_sk#15, s_state#16] +Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] +(17) CometColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#15, s_state#17] -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] +(18) BroadcastExchange +Input [2]: [s_store_sk#15, s_state#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] +(20) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] (21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) (23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] + +(24) CometColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#18, i_item_id#20] + +(25) BroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(26) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 5] +Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] + +(28) HashAggregate [codegen id : 5] +Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] +Keys [2]: [i_item_id#21, s_state#22] +Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] +Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] +Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(29) CometColumnarExchange +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 6] +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(31) HashAggregate [codegen id : 6] +Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Keys [2]: [i_item_id#21, s_state#22] +Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] +Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] +Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) +(33) CometFilter +Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) + +(34) CometColumnarToRow [codegen id : 11] +Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] +(35) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#61] -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_cdemo_sk#53] +Right keys [1]: [cd_demo_sk#61] +Join type: Inner +Join condition: None -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +(37) Project [codegen id : 11] +Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#61] -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] +(38) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#62] -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#62] +Join type: Inner +Join condition: None -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +(40) Project [codegen id : 11] +Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] +Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#62] -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight +(42) CometFilter +Input [2]: [s_store_sk#63, s_state#64] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#64, 2, true, false, true) = TN) AND isnotnull(s_store_sk#63)) (43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Input [2]: [s_store_sk#63, s_state#64] +Arguments: [s_store_sk#63], [s_store_sk#63] + +(44) CometColumnarToRow [codegen id : 9] +Input [1]: [s_store_sk#63] + +(45) BroadcastExchange +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#63] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 11] +Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] +Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#63] + +(48) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#65, i_item_id#20] + +(49) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#65] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 11] +Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] +Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#20] + +(51) HashAggregate [codegen id : 11] +Input [5]: [i_item_id#20, agg1#66, agg2#67, agg3#68, agg4#69] +Keys [1]: [i_item_id#20] +Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] +Aggregate Attributes [8]: [sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] +Results [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] + +(52) CometColumnarExchange +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(53) CometColumnarToRow [codegen id : 12] +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] + +(54) HashAggregate [codegen id : 12] +Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] +Keys [1]: [i_item_id#20] +Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] +Aggregate Attributes [4]: [avg(agg1#66)#86, avg(UnscaledValue(agg2#67))#87, avg(UnscaledValue(agg3#68))#88, avg(UnscaledValue(agg4#69))#89] +Results [7]: [i_item_id#20, null AS s_state#90, 1 AS g_state#91, avg(agg1#66)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#67))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#68))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#69))#89 / 100.0) as decimal(11,6)) AS agg4#95] + +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter -Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) +(56) CometFilter +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) + +(57) CometColumnarToRow [codegen id : 17] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#71] +(58) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#105] -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [cd_demo_sk#71] -Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight +(59) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#105] +Join type: Inner +Join condition: None -(54) CometProject -Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +(60) Project [codegen id : 17] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#105] -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#72] +(61) ReusedExchange [Reuses operator id: 85] +Output [1]: [d_date_sk#106] -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [d_date_sk#72] -Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight +(62) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#106] +Join type: Inner +Join condition: None -(57) CometProject -Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +(63) Project [codegen id : 17] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#73] +(64) ReusedExchange [Reuses operator id: 45] +Output [1]: [s_store_sk#107] -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [s_store_sk#73] -Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight +(65) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#107] +Join type: Inner +Join condition: None -(60) CometProject -Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] -Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +(66) Project [codegen id : 17] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#107] -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#74] +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(62) CometFilter -Input [1]: [i_item_sk#74] -Condition : isnotnull(i_item_sk#74) +(68) CometFilter +Input [1]: [i_item_sk#108] +Condition : isnotnull(i_item_sk#108) -(63) CometBroadcastExchange -Input [1]: [i_item_sk#74] -Arguments: [i_item_sk#74] +(69) CometColumnarToRow [codegen id : 16] +Input [1]: [i_item_sk#108] -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [i_item_sk#74] -Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight +(70) BroadcastExchange +Input [1]: [i_item_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(65) CometProject -Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] -Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] +(71) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#108] +Join type: Inner +Join condition: None -(66) CometHashAggregate -Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] +(72) Project [codegen id : 17] +Output [4]: [ss_quantity#99 AS agg1#109, ss_list_price#100 AS agg2#110, ss_coupon_amt#102 AS agg3#111, ss_sales_price#101 AS agg4#112] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#108] + +(73) HashAggregate [codegen id : 17] +Input [4]: [agg1#109, agg2#110, agg3#111, agg4#112] Keys: [] -Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] +Functions [4]: [partial_avg(agg1#109), partial_avg(UnscaledValue(agg2#110)), partial_avg(UnscaledValue(agg3#111)), partial_avg(UnscaledValue(agg4#112))] +Aggregate Attributes [8]: [sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] + +(74) CometColumnarExchange +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(67) CometExchange -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(75) CometColumnarToRow [codegen id : 18] +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] -(68) CometHashAggregate -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] +(76) HashAggregate [codegen id : 18] +Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] Keys: [] -Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] +Functions [4]: [avg(agg1#109), avg(UnscaledValue(agg2#110)), avg(UnscaledValue(agg3#111)), avg(UnscaledValue(agg4#112))] +Aggregate Attributes [4]: [avg(agg1#109)#129, avg(UnscaledValue(agg2#110))#130, avg(UnscaledValue(agg3#111))#131, avg(UnscaledValue(agg4#112))#132] +Results [7]: [null AS i_item_id#133, null AS s_state#134, 1 AS g_state#135, avg(agg1#109)#129 AS agg1#136, cast((avg(UnscaledValue(agg2#110))#130 / 100.0) as decimal(11,6)) AS agg2#137, cast((avg(UnscaledValue(agg3#111))#131 / 100.0) as decimal(11,6)) AS agg3#138, cast((avg(UnscaledValue(agg4#112))#132 / 100.0) as decimal(11,6)) AS agg4#139] + +(77) Union -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] -Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] +(78) RowToColumnar +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +(79) CometTakeOrderedAndProject +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#21 ASC NULLS FIRST,s_state#22 ASC NULLS FIRST], output=[i_item_id#21,s_state#22,g_state#47,agg1#48,agg2#49,agg3#50,agg4#51]), [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51], 100, 0, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +(80) CometColumnarToRow [codegen id : 19] +Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#140] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) +(82) CometFilter +Input [2]: [d_date_sk#14, d_year#140] +Condition : ((isnotnull(d_year#140) AND (d_year#140 = 1998)) AND isnotnull(d_date_sk#14)) -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] +(83) CometProject +Input [2]: [d_date_sk#14, d_year#140] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(85) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt index affbc9ef37..c26488e3c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt @@ -1,99 +1,118 @@ CometColumnarToRow +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 + +- RowToColumnar + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 48 out of 95 eligible operators (50%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 0d18ca5626..293a5fc379 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -1,83 +1,126 @@ -WholeStageCodegen (1) +WholeStageCodegen (19) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) + RowToColumnar + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,s_state] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] - CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [cd_demo_sk] #3 - ReusedExchange [d_date_sk] #4 - ReusedExchange [s_store_sk] #8 - CometBroadcastExchange [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (17) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [s_store_sk] #7 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt index 50af5f4f75..0447825143 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt @@ -1,37 +1,39 @@ == Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +* CometColumnarToRow (35) ++- CometSort (34) + +- CometColumnarExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * Filter (25) + : +- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (30) + +- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -46,177 +48,184 @@ ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(4) ReusedExchange [Reuses operator id: 40] +Output [1]: [d_date_sk#7] -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) +(8) CometFilter +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] +(9) CometProject +Input [2]: [s_store_sk#8, s_county#9] +Arguments: [s_store_sk#8], [s_store_sk#8] -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] +(10) CometColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#8] -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight +(11) BroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#8] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +(13) Project [codegen id : 4] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) +(15) CometFilter +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) + +(16) CometProject +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] +(17) CometColumnarToRow [codegen id : 3] +Input [1]: [hd_demo_sk#10] -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] +(18) BroadcastExchange +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#10] +Join type: Inner +Join condition: None -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] +(20) Project [codegen id : 4] +Output [2]: [ss_customer_sk#1, ss_ticket_number#4] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) CometHashAggregate +(21) HashAggregate [codegen id : 4] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +(23) CometColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(24) HashAggregate [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(24) CometFilter +(25) Filter [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter +(27) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(27) CometProject +(28) CometProject Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(28) CometBroadcastExchange +(29) CometColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] + +(30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None -(30) CometProject +(32) Project [codegen id : 6] +Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometExchange +(33) CometColumnarExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort +(34) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] -(33) CometColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 7] Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) +BroadcastExchange (40) ++- * CometColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#27, d_dom#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(37) CometFilter +Input [3]: [d_date_sk#7, d_year#27, d_dom#28] +Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +(38) CometProject +Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..7cb35600f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 20 out of 37 eligible operators (54%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt index d57afd4fcf..568e86c9fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt @@ -1,43 +1,59 @@ -WholeStageCodegen (1) +WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt index 50af5f4f75..0447825143 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt @@ -1,37 +1,39 @@ == Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +* CometColumnarToRow (35) ++- CometSort (34) + +- CometColumnarExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * Filter (25) + : +- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (30) + +- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -46,177 +48,184 @@ ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(4) ReusedExchange [Reuses operator id: 40] +Output [1]: [d_date_sk#7] -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) +(8) CometFilter +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] +(9) CometProject +Input [2]: [s_store_sk#8, s_county#9] +Arguments: [s_store_sk#8], [s_store_sk#8] -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] +(10) CometColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#8] -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight +(11) BroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#8] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +(13) Project [codegen id : 4] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) +(15) CometFilter +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) + +(16) CometProject +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] +(17) CometColumnarToRow [codegen id : 3] +Input [1]: [hd_demo_sk#10] -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] +(18) BroadcastExchange +Input [1]: [hd_demo_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#10] +Join type: Inner +Join condition: None -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] +(20) Project [codegen id : 4] +Output [2]: [ss_customer_sk#1, ss_ticket_number#4] +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) CometHashAggregate +(21) HashAggregate [codegen id : 4] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +(23) CometColumnarToRow [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(24) HashAggregate [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#16] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(24) CometFilter +(25) Filter [codegen id : 6] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter +(27) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(27) CometProject +(28) CometProject Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(28) CometBroadcastExchange +(29) CometColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] + +(30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None -(30) CometProject +(32) Project [codegen id : 6] +Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometExchange +(33) CometColumnarExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(32) CometSort +(34) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] -(33) CometColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 7] Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) +BroadcastExchange (40) ++- * CometColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#27, d_dom#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(37) CometFilter +Input [3]: [d_date_sk#7, d_year#27, d_dom#28] +Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +(38) CometProject +Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(38) BroadcastExchange +(40) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt index 3d9d23d2cc..7cb35600f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 20 out of 37 eligible operators (54%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index d57afd4fcf..568e86c9fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -1,43 +1,59 @@ -WholeStageCodegen (1) +WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] - CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) + CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt index b43ed9b192..f08a74ef15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt @@ -1,51 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) +* CometColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * CometColumnarToRow (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * CometColumnarToRow (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (31) + : +- * CometColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (34) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,217 +59,214 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] +(6) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#9] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: LeftSemi +Join condition: None -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight +(13) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#13] -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#10] +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(16) BroadcastExchange +Input [1]: [ws_bill_customer_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] +Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] ReadSchema: struct -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] +(19) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight +(20) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#17] -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#14] +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(23) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#14] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(25) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(26) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +(28) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] +(29) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] +(30) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#20] -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(31) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] +(33) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#4, ca_state#20] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +(35) CometFilter +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Condition : isnotnull(cd_demo_sk#21) -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) CometProject +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) CometColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 5] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(40) Project [codegen id : 9] +Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] + +(41) HashAggregate [codegen id : 9] +Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] + +(42) CometColumnarExchange +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +(43) CometColumnarToRow [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(44) HashAggregate [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] +Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] +Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +(45) RowToColumnar +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] +(46) CometTakeOrderedAndProject +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +(47) CometColumnarToRow [codegen id : 11] +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) +- * CometColumnarToRow (51) +- CometProject (50) @@ -278,18 +275,18 @@ BroadcastExchange (52) (48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Output [3]: [d_date_sk#9, d_year#77, d_qoy#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 2002)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#9)) (50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] Arguments: [d_date_sk#9], [d_date_sk#9] (51) CometColumnarToRow [codegen id : 1] @@ -297,10 +294,10 @@ Input [1]: [d_date_sk#9] (52) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index be2d144fe2..8978af8750 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,69 @@ - TakeOrderedAndProject [COMET: ] -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 54 eligible operators (46%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt index fea6a32741..9e3618d14c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt @@ -1,73 +1,81 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) +WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastExchange #7 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt index b43ed9b192..f08a74ef15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt @@ -1,51 +1,51 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) +* CometColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- RowToColumnar (45) + +- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * CometColumnarToRow (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * CometColumnarToRow (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (31) + : +- * CometColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (34) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,217 +59,214 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] +(6) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#9] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: LeftSemi +Join condition: None -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight +(13) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#13] -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#10] +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(16) BroadcastExchange +Input [1]: [ws_bill_customer_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] +Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) Join condition: None -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] ReadSchema: struct -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] +(19) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight +(20) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#17] -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#14] +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(23) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#14] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(25) Filter [codegen id : 9] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(26) Project [codegen id : 9] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +(28) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] +(29) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] +(30) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#20] -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(31) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 5] +(32) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] +(33) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#4, ca_state#20] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +(35) CometFilter +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Condition : isnotnull(cd_demo_sk#21) -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) CometProject +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) CometColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 5] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(40) Project [codegen id : 9] +Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] + +(41) HashAggregate [codegen id : 9] +Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] + +(42) CometColumnarExchange +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +(43) CometColumnarToRow [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(44) HashAggregate [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] +Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] +Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +(45) RowToColumnar +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] +(46) CometTakeOrderedAndProject +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +(47) CometColumnarToRow [codegen id : 11] +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) +- * CometColumnarToRow (51) +- CometProject (50) @@ -278,18 +275,18 @@ BroadcastExchange (52) (48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Output [3]: [d_date_sk#9, d_year#77, d_qoy#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 2002)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#9)) (50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] Arguments: [d_date_sk#9], [d_date_sk#9] (51) CometColumnarToRow [codegen id : 1] @@ -297,10 +294,10 @@ Input [1]: [d_date_sk#9] (52) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt index be2d144fe2..8978af8750 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt @@ -1,63 +1,69 @@ - TakeOrderedAndProject [COMET: ] -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : : : +- CometColumnarToRow - : : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 54 eligible operators (46%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index fea6a32741..9e3618d14c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -1,73 +1,81 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) +WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastExchange #7 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt index e22a5c492c..0c6a3e9fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) +* CometColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * CometColumnarToRow (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * CometColumnarToRow (17) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * CometColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -52,216 +57,233 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(6) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#7] -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] ReadSchema: struct -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#11] -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customsk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] ReadSchema: struct -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] +(17) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight +(18) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#16] -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customsk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] +(21) Union -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight +(22) BroadcastExchange +Input [1]: [customsk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) CometProject +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customsk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) + +(27) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] +(28) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#20] -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] +(29) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] +(31) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, ca_state#20] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +(33) CometFilter +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Condition : isnotnull(cd_demo_sk#21) -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometProject +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(35) CometColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight +(36) BroadcastExchange +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#21] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +(38) Project [codegen id : 9] +Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(39) HashAggregate [codegen id : 9] +Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +(40) CometColumnarExchange +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +(41) CometColumnarToRow [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +(42) HashAggregate [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] +Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] +Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] + +(43) RowToColumnar +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] + +(44) CometTakeOrderedAndProject +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] + +(45) CometColumnarToRow [codegen id : 11] +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#77, d_qoy#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) +(47) CometFilter +Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] +Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 1999)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +(48) CometProject +Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..fbfef118e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -1,56 +1,67 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt index 69e8d4868c..6614ba7532 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt @@ -1,52 +1,78 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customsk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt index e22a5c492c..0c6a3e9fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) +* CometColumnarToRow (45) ++- CometTakeOrderedAndProject (44) + +- RowToColumnar (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * CometColumnarToRow (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * CometColumnarToRow (12) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * CometColumnarToRow (17) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * CometColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -52,216 +57,233 @@ ReadSchema: struct -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] +(5) CometColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(6) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#7] -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(9) CometProject +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(10) CometBroadcastExchange +(9) BroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight +(10) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] ReadSchema: struct -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] +(12) CometColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#11] -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customsk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] ReadSchema: struct -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] +(17) CometColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight +(18) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#16] -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customsk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] +(21) Union -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight +(22) BroadcastExchange +Input [1]: [customsk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) CometProject +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customsk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) + +(27) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] +(28) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#20] -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] +(29) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] +(31) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, ca_state#20] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +(33) CometFilter +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Condition : isnotnull(cd_demo_sk#21) -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometProject +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(35) CometColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight +(36) BroadcastExchange +Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#21] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +(38) Project [codegen id : 9] +Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(39) HashAggregate [codegen id : 9] +Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +(40) CometColumnarExchange +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +(41) CometColumnarToRow [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +(42) HashAggregate [codegen id : 10] +Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] +Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] +Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] + +(43) RowToColumnar +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] + +(44) CometTakeOrderedAndProject +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] + +(45) CometColumnarToRow [codegen id : 11] +Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] ===== Subqueries ===== -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#77, d_qoy#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) +(47) CometFilter +Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] +Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 1999)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +(48) CometProject +Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt index 2cdc75e15e..fbfef118e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt @@ -1,56 +1,67 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 + +- RowToColumnar + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index 69e8d4868c..6614ba7532 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -1,52 +1,78 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometProject [c_current_cdemo_sk,ca_state] - CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] - CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometProject [cs_ship_customer_sk] [customsk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + RowToColumnar + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customsk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt index 2562693a01..68c5730e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt @@ -1,47 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- RowToColumnar (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometExchange (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- Union (39) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- ReusedExchange (25) + +- * HashAggregate (38) + +- * CometColumnarToRow (37) + +- CometColumnarExchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- * CometColumnarToRow (33) + +- ReusedExchange (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -56,220 +63,252 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(4) ReusedExchange [Reuses operator id: 55] +Output [1]: [d_date_sk#7] -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) +(8) CometFilter +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Condition : isnotnull(i_item_sk#8) -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] +(9) CometProject +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] +(10) CometColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#8, i_class#11, i_category#12] -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight +(11) BroadcastExchange +Input [3]: [i_item_sk#8, i_class#11, i_category#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#8] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] +(13) Project [codegen id : 4] +Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) +(15) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] +(16) CometProject +Input [2]: [s_store_sk#13, s_state#14] +Arguments: [s_store_sk#13], [s_store_sk#13] -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] +(17) CometColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#13] -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight +(18) BroadcastExchange +Input [1]: [s_store_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +(20) Project [codegen id : 4] +Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] +(21) HashAggregate [codegen id : 4] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum#15, sum#16] +Results [4]: [i_category#12, i_class#11, sum#17, sum#18] -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] +(22) CometColumnarExchange +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] +Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(23) CometColumnarToRow [codegen id : 5] +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] +(24) HashAggregate [codegen id : 5] +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] + +(25) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#12, i_class#11, sum#27, sum#28] + +(26) CometColumnarToRow [codegen id : 10] +Input [4]: [i_category#12, i_class#11, sum#27, sum#28] + +(27) HashAggregate [codegen id : 10] +Input [4]: [i_category#12, i_class#11, sum#27, sum#28] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] + +(28) HashAggregate [codegen id : 10] +Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] +Keys [1]: [i_category#12] +Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] +Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] +Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] + +(29) CometColumnarExchange +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] +Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 11] +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] + +(31) HashAggregate [codegen id : 11] +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] +Keys [1]: [i_category#12] +Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] +Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] +Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] + +(32) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#12, i_class#11, sum#50, sum#51] + +(33) CometColumnarToRow [codegen id : 16] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] + +(34) HashAggregate [codegen id : 16] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] + +(35) HashAggregate [codegen id : 16] +Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] +Keys: [] +Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] +Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] +(36) CometColumnarExchange +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] +(37) CometColumnarToRow [codegen id : 17] +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(31) CometHashAggregate -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +(38) HashAggregate [codegen id : 17] +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] +Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] +Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as decimal(38,20)) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] -(32) CometExchange -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(39) Union -(33) CometHashAggregate -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +(40) HashAggregate [codegen id : 18] +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Functions: [] +Aggregate Attributes: [] +Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -(34) CometUnion -Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] -Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] +(41) CometColumnarExchange +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +(42) CometHashAggregate +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Functions: [] -(36) CometExchange -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(43) CometExchange +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] +(44) CometSort +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] -(38) CometExchange -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(45) CometColumnarToRow [codegen id : 19] +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -(39) CometSort -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] +(46) Window +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +(47) Project [codegen id : 20] +Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] -(41) Window -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] +(48) RowToColumnar +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -(42) Project [codegen id : 2] -Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] +(49) CometTakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#26 DESC NULLS LAST,CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST,rank_within_parent#73 ASC NULLS FIRST], output=[gross_margin#21,i_category#22,i_class#23,lochierarchy#26,rank_within_parent#73]), [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73], 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -(43) TakeOrderedAndProject -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +(50) CometColumnarToRow [codegen id : 21] +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#74] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) +(52) CometFilter +Input [2]: [d_date_sk#7, d_year#74] +Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] +(53) CometProject +Input [2]: [d_date_sk#7, d_year#74] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(48) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index dee6669b47..c50592042c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -1,105 +1,124 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 52 out of 99 eligible operators (52%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt index e91b278c14..1501a1fbd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt @@ -1,55 +1,85 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (21) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (20) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (11) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (10) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (16) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt index 2562693a01..68c5730e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt @@ -1,47 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- RowToColumnar (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometExchange (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- Union (39) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- ReusedExchange (25) + +- * HashAggregate (38) + +- * CometColumnarToRow (37) + +- CometColumnarExchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- * CometColumnarToRow (33) + +- ReusedExchange (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -56,220 +63,252 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] +(4) ReusedExchange [Reuses operator id: 55] +Output [1]: [d_date_sk#7] -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 4] +Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) +(8) CometFilter +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Condition : isnotnull(i_item_sk#8) -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] +(9) CometProject +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] +(10) CometColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#8, i_class#11, i_category#12] -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight +(11) BroadcastExchange +Input [3]: [i_item_sk#8, i_class#11, i_category#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#8] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] +(13) Project [codegen id : 4] +Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) +(15) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] +(16) CometProject +Input [2]: [s_store_sk#13, s_state#14] +Arguments: [s_store_sk#13], [s_store_sk#13] -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] +(17) CometColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#13] -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight +(18) BroadcastExchange +Input [1]: [s_store_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +(20) Project [codegen id : 4] +Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] +(21) HashAggregate [codegen id : 4] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum#15, sum#16] +Results [4]: [i_category#12, i_class#11, sum#17, sum#18] -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] +(22) CometColumnarExchange +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] +Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(23) CometColumnarToRow [codegen id : 5] +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] +(24) HashAggregate [codegen id : 5] +Input [4]: [i_category#12, i_class#11, sum#17, sum#18] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] + +(25) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#12, i_class#11, sum#27, sum#28] + +(26) CometColumnarToRow [codegen id : 10] +Input [4]: [i_category#12, i_class#11, sum#27, sum#28] + +(27) HashAggregate [codegen id : 10] +Input [4]: [i_category#12, i_class#11, sum#27, sum#28] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] + +(28) HashAggregate [codegen id : 10] +Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] +Keys [1]: [i_category#12] +Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] +Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] +Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] + +(29) CometColumnarExchange +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] +Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 11] +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] + +(31) HashAggregate [codegen id : 11] +Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] +Keys [1]: [i_category#12] +Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] +Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] +Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] + +(32) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#12, i_class#11, sum#50, sum#51] + +(33) CometColumnarToRow [codegen id : 16] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] + +(34) HashAggregate [codegen id : 16] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] + +(35) HashAggregate [codegen id : 16] +Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] +Keys: [] +Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] +Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] +(36) CometColumnarExchange +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] +(37) CometColumnarToRow [codegen id : 17] +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(31) CometHashAggregate -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +(38) HashAggregate [codegen id : 17] +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] +Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] +Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as decimal(38,20)) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] -(32) CometExchange -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(39) Union -(33) CometHashAggregate -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +(40) HashAggregate [codegen id : 18] +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Functions: [] +Aggregate Attributes: [] +Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -(34) CometUnion -Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] -Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] +(41) CometColumnarExchange +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +(42) CometHashAggregate +Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Functions: [] -(36) CometExchange -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(43) CometExchange +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] +(44) CometSort +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] -(38) CometExchange -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(45) CometColumnarToRow [codegen id : 19] +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -(39) CometSort -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] +(46) Window +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +(47) Project [codegen id : 20] +Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] -(41) Window -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] +(48) RowToColumnar +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -(42) Project [codegen id : 2] -Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] +(49) CometTakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#26 DESC NULLS LAST,CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST,rank_within_parent#73 ASC NULLS FIRST], output=[gross_margin#21,i_category#22,i_class#23,lochierarchy#26,rank_within_parent#73]), [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73], 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -(43) TakeOrderedAndProject -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +(50) CometColumnarToRow [codegen id : 21] +Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#74] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) +(52) CometFilter +Input [2]: [d_date_sk#7, d_year#74] +Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] +(53) CometProject +Input [2]: [d_date_sk#7, d_year#74] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(48) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt index dee6669b47..c50592042c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt @@ -1,105 +1,124 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 52 out of 99 eligible operators (52%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index e91b278c14..1501a1fbd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -1,55 +1,85 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (21) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (20) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (11) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (10) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (16) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt index 7b1f860f32..0c7c2a3bce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt @@ -1,52 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (53) ++- CometTakeOrderedAndProject (52) + +- RowToColumnar (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * CometColumnarToRow (26) + : : +- CometSort (25) + : : +- CometColumnarExchange (24) + : : +- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- Window (38) + : +- * CometColumnarToRow (37) + : +- CometSort (36) + : +- CometColumnarExchange (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + +- BroadcastExchange (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -64,7 +69,10 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#4, i_category#5] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -72,46 +80,39 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight +(8) BroadcastExchange +Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 4] +Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] @@ -124,166 +125,188 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometBroadcastExchange +(16) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight +(17) BroadcastExchange +Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) CometProject +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#7] +Right keys [1]: [s_store_sk#14] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -(19) CometHashAggregate +(20) HashAggregate [codegen id : 4] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] +Aggregate Attributes [1]: [sum#17] +Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] + +(21) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] +(23) HashAggregate [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] +Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(24) CometColumnarExchange +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) CometSort +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(26) CometColumnarToRow [codegen id : 6] +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(27) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(28) Filter [codegen id : 7] +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +(29) Window +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) +(30) Filter [codegen id : 22] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] +Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(31) Project [codegen id : 22] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] +(32) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] +(33) CometColumnarToRow [codegen id : 12] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(34) HashAggregate [codegen id : 12] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] +Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(35) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +(36) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(37) CometColumnarToRow [codegen id : 13] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +(38) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +(39) Project [codegen id : 14] +Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] +Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +(40) BroadcastExchange +Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +(42) Project [codegen id : 22] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] +Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +(43) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(44) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +(45) CometColumnarToRow [codegen id : 20] +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(46) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(47) Project [codegen id : 21] +Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] +Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(48) BroadcastExchange +Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +(49) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] +Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(50) Project [codegen id : 22] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] + +(51) RowToColumnar +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] + +(52) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#23,sum_sales#20,psum#42,nsum#43]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43], 100, 0, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(53) CometColumnarToRow [codegen id : 23] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (57) ++- * CometColumnarToRow (56) + +- CometFilter (55) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(55) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index 11fbf5bd11..d48e51fba0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -1,105 +1,122 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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 - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt index a62c33ecc1..329fe73435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt @@ -1,71 +1,91 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] +WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + RowToColumnar + WholeStageCodegen (22) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (7) + Filter [d_year] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt index 7b1f860f32..0c7c2a3bce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt @@ -1,52 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (53) ++- CometTakeOrderedAndProject (52) + +- RowToColumnar (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * CometColumnarToRow (26) + : : +- CometSort (25) + : : +- CometColumnarExchange (24) + : : +- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- Window (38) + : +- * CometColumnarToRow (37) + : +- CometSort (36) + : +- CometColumnarExchange (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + +- BroadcastExchange (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -64,7 +69,10 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#4, i_category#5] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -72,46 +80,39 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight +(8) BroadcastExchange +Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 4] +Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] @@ -124,166 +125,188 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometBroadcastExchange +(16) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight +(17) BroadcastExchange +Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) CometProject +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#7] +Right keys [1]: [s_store_sk#14] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -(19) CometHashAggregate +(20) HashAggregate [codegen id : 4] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] +Aggregate Attributes [1]: [sum#17] +Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] + +(21) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] +(23) HashAggregate [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] +Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(24) CometColumnarExchange +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) CometSort +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(26) CometColumnarToRow [codegen id : 6] +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(27) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(28) Filter [codegen id : 7] +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +(29) Window +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) +(30) Filter [codegen id : 22] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] +Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(31) Project [codegen id : 22] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] +(32) ReusedExchange [Reuses operator id: 21] +Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] +(33) CometColumnarToRow [codegen id : 12] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(34) HashAggregate [codegen id : 12] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] +Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(35) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +(36) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(37) CometColumnarToRow [codegen id : 13] +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +(38) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +(39) Project [codegen id : 14] +Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] +Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +(40) BroadcastExchange +Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +(42) Project [codegen id : 22] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] +Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +(43) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(44) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +(45) CometColumnarToRow [codegen id : 20] +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(46) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(47) Project [codegen id : 21] +Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] +Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(48) BroadcastExchange +Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +(49) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] +Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(50) Project [codegen id : 22] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] + +(51) RowToColumnar +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] + +(52) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#23,sum_sales#20,psum#42,nsum#43]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43], 100, 0, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(53) CometColumnarToRow [codegen id : 23] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (57) ++- * CometColumnarToRow (56) + +- CometFilter (55) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(55) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt index 11fbf5bd11..d48e51fba0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt @@ -1,105 +1,122 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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 - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index a62c33ecc1..329fe73435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -1,71 +1,91 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] +WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + RowToColumnar + WholeStageCodegen (22) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (7) + Filter [d_year] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt index 74702d596d..9b6b457306 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- Union (79) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -12,71 +12,77 @@ : +- Window (22) : +- * CometColumnarToRow (21) : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) + : +- CometColumnarExchange (19) + : +- * HashAggregate (18) + : +- * CometColumnarToRow (17) + : +- CometColumnarExchange (16) + : +- * HashAggregate (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildLeft (10) + : : :- BroadcastExchange (5) + : : : +- * CometColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (6) + : +- ReusedExchange (12) + :- * Project (52) + : +- * Filter (51) + : +- Window (50) + : +- * Sort (49) + : +- Window (48) + : +- * CometColumnarToRow (47) + : +- CometSort (46) + : +- CometColumnarExchange (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildLeft (36) + : : :- BroadcastExchange (31) + : : : +- * CometColumnarToRow (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- * CometColumnarToRow (35) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (32) + : +- ReusedExchange (38) + +- * Project (78) + +- * Filter (77) + +- Window (76) + +- * Sort (75) + +- Window (74) + +- * CometColumnarToRow (73) + +- CometSort (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- * Project (63) + : +- * BroadcastHashJoin Inner BuildLeft (62) + : :- BroadcastExchange (57) + : : +- * CometColumnarToRow (56) + : : +- CometProject (55) + : : +- CometFilter (54) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (53) + : +- * CometColumnarToRow (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (58) + +- ReusedExchange (64) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -95,368 +101,397 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) CometBroadcastExchange +(4) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(5) BroadcastExchange +Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] + +(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(7) CometProject +(8) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft +(9) CometColumnarToRow +Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [ws_order_number#2, ws_item_sk#1] +Right keys [2]: [wr_order_number#9, wr_item_sk#8] +Join type: Inner +Join condition: None -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] +(11) Project [codegen id : 3] +Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] +(12) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#13] -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight +(13) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(15) CometProject +(14) Project [codegen id : 3] +Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) CometHashAggregate +(15) HashAggregate [codegen id : 3] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] +Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(16) CometColumnarExchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(17) CometColumnarToRow [codegen id : 4] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +(18) HashAggregate [codegen id : 4] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] +(21) CometColumnarToRow [codegen id : 5] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 (24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) +(25) Filter [codegen id : 7] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +(26) Project [codegen id : 7] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct (28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) (29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +(30) CometColumnarToRow [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +(31) BroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +(33) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) + +(34) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(35) CometColumnarToRow +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(36) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [cs_order_number#37, cs_item_sk#36] +Right keys [2]: [cr_order_number#44, cr_item_sk#43] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 10] +Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(38) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#48] + +(39) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 10] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] + +(41) HashAggregate [codegen id : 10] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(42) CometColumnarExchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(43) CometColumnarToRow [codegen id : 11] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(44) HashAggregate [codegen id : 11] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(45) CometColumnarExchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(46) CometSort +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] + +(47) CometColumnarToRow [codegen id : 12] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] + +(48) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] + +(49) Sort [codegen id : 13] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 + +(50) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] + +(51) Filter [codegen id : 14] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) + +(52) Project [codegen id : 14] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) +(54) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) + +(55) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +(56) CometColumnarToRow [codegen id : 15] +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +(57) BroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(59) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) + +(60) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(61) CometColumnarToRow +Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(62) BroadcastHashJoin [codegen id : 17] +Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] +Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 17] +Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(64) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#83] + +(65) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] +Join type: Inner +Join condition: None + +(66) Project [codegen id : 17] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(67) HashAggregate [codegen id : 17] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(68) CometColumnarExchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(69) CometColumnarToRow [codegen id : 18] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(70) HashAggregate [codegen id : 18] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] + +(71) CometColumnarExchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(72) CometSort +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] + +(73) CometColumnarToRow [codegen id : 19] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] + +(74) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] + +(75) Sort [codegen id : 20] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 + +(76) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] + +(77) Filter [codegen id : 21] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) + +(78) Project [codegen id : 21] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] + +(79) Union + +(80) HashAggregate [codegen id : 22] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometColumnarExchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(82) CometHashAggregate +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(83) CometTakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(84) CometColumnarToRow [codegen id : 23] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) +(86) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +(87) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index 734b0ce7bb..1a76c9c7f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 87 eligible operators (48%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt index ca80833ee5..d836b433a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt @@ -1,110 +1,140 @@ -WholeStageCodegen (11) +WholeStageCodegen (23) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) + WholeStageCodegen (22) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (3) + WholeStageCodegen (7) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (2) + WholeStageCodegen (6) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (1) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 + CometColumnarExchange #2 + WholeStageCodegen (4) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (14) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (13) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (12) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) + CometColumnarExchange #6 + WholeStageCodegen (11) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (10) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (21) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (8) + WholeStageCodegen (20) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (7) + WholeStageCodegen (19) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange #9 + WholeStageCodegen (18) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (17) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 74702d596d..9b6b457306 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- Union (79) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -12,71 +12,77 @@ : +- Window (22) : +- * CometColumnarToRow (21) : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) + : +- CometColumnarExchange (19) + : +- * HashAggregate (18) + : +- * CometColumnarToRow (17) + : +- CometColumnarExchange (16) + : +- * HashAggregate (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildLeft (10) + : : :- BroadcastExchange (5) + : : : +- * CometColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (6) + : +- ReusedExchange (12) + :- * Project (52) + : +- * Filter (51) + : +- Window (50) + : +- * Sort (49) + : +- Window (48) + : +- * CometColumnarToRow (47) + : +- CometSort (46) + : +- CometColumnarExchange (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildLeft (36) + : : :- BroadcastExchange (31) + : : : +- * CometColumnarToRow (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- * CometColumnarToRow (35) + : : +- CometProject (34) + : : +- CometFilter (33) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (32) + : +- ReusedExchange (38) + +- * Project (78) + +- * Filter (77) + +- Window (76) + +- * Sort (75) + +- Window (74) + +- * CometColumnarToRow (73) + +- CometSort (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- * Project (63) + : +- * BroadcastHashJoin Inner BuildLeft (62) + : :- BroadcastExchange (57) + : : +- * CometColumnarToRow (56) + : : +- CometProject (55) + : : +- CometFilter (54) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (53) + : +- * CometColumnarToRow (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (58) + +- ReusedExchange (64) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -95,368 +101,397 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) CometBroadcastExchange +(4) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(5) BroadcastExchange +Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] + +(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(6) CometFilter +(7) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(7) CometProject +(8) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft +(9) CometColumnarToRow +Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [ws_order_number#2, ws_item_sk#1] +Right keys [2]: [wr_order_number#9, wr_item_sk#8] +Join type: Inner +Join condition: None -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] +(11) Project [codegen id : 3] +Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] +(12) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#13] -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight +(13) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(15) CometProject +(14) Project [codegen id : 3] +Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) CometHashAggregate +(15) HashAggregate [codegen id : 3] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] +Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(16) CometColumnarExchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(17) CometColumnarToRow [codegen id : 4] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +(18) HashAggregate [codegen id : 4] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(19) CometColumnarExchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] +(21) CometColumnarToRow [codegen id : 5] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 (24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) +(25) Filter [codegen id : 7] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +(26) Project [codegen id : 7] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct (28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) (29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +(30) CometColumnarToRow [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +(31) BroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +(33) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) + +(34) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(35) CometColumnarToRow +Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(36) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [cs_order_number#37, cs_item_sk#36] +Right keys [2]: [cr_order_number#44, cr_item_sk#43] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 10] +Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(38) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#48] + +(39) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 10] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] + +(41) HashAggregate [codegen id : 10] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(42) CometColumnarExchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(43) CometColumnarToRow [codegen id : 11] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(44) HashAggregate [codegen id : 11] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(45) CometColumnarExchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(46) CometSort +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] + +(47) CometColumnarToRow [codegen id : 12] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] + +(48) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] + +(49) Sort [codegen id : 13] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 + +(50) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] + +(51) Filter [codegen id : 14] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) + +(52) Project [codegen id : 14] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) +(54) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) + +(55) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +(56) CometColumnarToRow [codegen id : 15] +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +(57) BroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(59) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) + +(60) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(61) CometColumnarToRow +Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(62) BroadcastHashJoin [codegen id : 17] +Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] +Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 17] +Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(64) ReusedExchange [Reuses operator id: 89] +Output [1]: [d_date_sk#83] + +(65) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] +Join type: Inner +Join condition: None + +(66) Project [codegen id : 17] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(67) HashAggregate [codegen id : 17] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(68) CometColumnarExchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(69) CometColumnarToRow [codegen id : 18] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(70) HashAggregate [codegen id : 18] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] + +(71) CometColumnarExchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(72) CometSort +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] + +(73) CometColumnarToRow [codegen id : 19] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] + +(74) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] + +(75) Sort [codegen id : 20] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 + +(76) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] + +(77) Filter [codegen id : 21] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) + +(78) Project [codegen id : 21] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] + +(79) Union + +(80) HashAggregate [codegen id : 22] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometColumnarExchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(82) CometHashAggregate +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(83) CometTakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(84) CometColumnarToRow [codegen id : 23] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) +(86) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +(87) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt index 734b0ce7bb..1a76c9c7f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt @@ -11,31 +11,35 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 87 eligible operators (48%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index ca80833ee5..d836b433a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -1,110 +1,140 @@ -WholeStageCodegen (11) +WholeStageCodegen (23) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) + WholeStageCodegen (22) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (3) + WholeStageCodegen (7) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (2) + WholeStageCodegen (6) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (1) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 + CometColumnarExchange #2 + WholeStageCodegen (4) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (14) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (13) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (12) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) + CometColumnarExchange #6 + WholeStageCodegen (11) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (10) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (21) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (8) + WholeStageCodegen (20) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (7) + WholeStageCodegen (19) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange #9 + WholeStageCodegen (18) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (17) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt index f8a95a6db8..204830d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt @@ -1,80 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) +* CometColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- RowToColumnar (77) + +- * Filter (76) + +- * HashAggregate (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- Window (65) + : +- * CometColumnarToRow (64) + : +- CometSort (63) + : +- CometExchange (62) + : +- CometProject (61) + : +- CometFilter (60) + : +- CometSortMergeJoin (59) + : :- CometSort (29) + : : +- CometColumnarExchange (28) + : : +- * HashAggregate (27) + : : +- * CometColumnarToRow (26) + : : +- CometColumnarExchange (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (15) + : : : +- Window (14) + : : : +- * CometColumnarToRow (13) + : : : +- CometSort (12) + : : : +- CometColumnarExchange (11) + : : : +- * HashAggregate (10) + : : : +- * CometColumnarToRow (9) + : : : +- CometColumnarExchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * CometColumnarToRow (18) + : : +- CometSort (17) + : : +- ReusedExchange (16) + : +- CometSort (58) + : +- CometColumnarExchange (57) + : +- * HashAggregate (56) + : +- * CometColumnarToRow (55) + : +- CometColumnarExchange (54) + : +- * HashAggregate (53) + : +- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (44) + : : +- Window (43) + : : +- * CometColumnarToRow (42) + : : +- CometSort (41) + : : +- CometColumnarExchange (40) + : : +- * HashAggregate (39) + : : +- * CometColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * CometColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) + : : +- ReusedExchange (33) + : +- BroadcastExchange (50) + : +- * Project (49) + : +- Window (48) + : +- * CometColumnarToRow (47) + : +- CometSort (46) + : +- ReusedExchange (45) + +- BroadcastExchange (71) + +- * Project (70) + +- Window (69) + +- * CometColumnarToRow (68) + +- CometSort (67) + +- ReusedExchange (66) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -89,360 +92,372 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] +(3) CometColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] +(4) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#5, d_date#6] -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 2] +Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) CometHashAggregate +(7) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum#7] +Results [3]: [ws_item_sk#1, d_date#6, sum#8] + +(8) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometExchange +(9) CometColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometHashAggregate +(10) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(11) CometColumnarExchange +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(12) CometSort +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +(13) CometColumnarToRow [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(14) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] +(15) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(16) ReusedExchange [Reuses operator id: 11] +Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] +(17) CometSort +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(18) CometColumnarToRow [codegen id : 8] +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +(19) Window +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] +(20) Project [codegen id : 9] +Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] +Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] +(21) BroadcastExchange +Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] +(22) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#16] Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Join condition: (rk#12 >= rk#15) + +(23) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] + +(24) HashAggregate [codegen id : 10] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [partial_sum(sumws#17)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(25) CometColumnarExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(26) CometColumnarToRow [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(27) HashAggregate [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [sum(sumws#17)] +Aggregate Attributes [1]: [sum(sumws#17)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] + +(28) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(29) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) +(31) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) + +(32) CometColumnarToRow [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] + +(33) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#28, d_date#29] -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] +(34) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight +(35) Project [codegen id : 13] +Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] +(36) HashAggregate [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [ss_item_sk#24, d_date#29, sum#31] -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] +(37) CometColumnarExchange +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(38) CometColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] +(39) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] +Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(40) CometColumnarExchange +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(41) CometSort +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +(42) CometColumnarToRow [codegen id : 15] +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] +(43) Window +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] +(44) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] +Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(45) ReusedExchange [Reuses operator id: 40] +Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] +(46) CometSort +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(47) CometColumnarToRow [codegen id : 19] +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] +(48) Window +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] +(49) Project [codegen id : 20] +Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] +Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] +(50) BroadcastExchange +Input [3]: [item_sk#39, sumss#40, rk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] +(51) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [item_sk#33] +Right keys [1]: [item_sk#39] Join type: Inner -Join condition: (rk#32 >= rk#35) +Join condition: (rk#35 >= rk#38) -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] +(52) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +(53) HashAggregate [codegen id : 21] +Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [partial_sum(sumss#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(54) CometColumnarExchange +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +(55) CometColumnarToRow [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] +(56) HashAggregate [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [sum(sumss#40)] +Aggregate Attributes [1]: [sum(sumss#40)#45] +Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(57) CometColumnarExchange +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(58) CometSort +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter +(59) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#23] +Right output [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) +(60) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] +(61) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(62) CometExchange +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] +(63) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(64) CometColumnarToRow [codegen id : 23] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] +(65) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(66) ReusedExchange [Reuses operator id: 62] +Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] +(67) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(68) CometColumnarToRow [codegen id : 46] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] +(69) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] +(70) Project [codegen id : 47] +Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] +Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] +(71) BroadcastExchange +Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] +(72) BroadcastHashJoin [codegen id : 48] +Left keys [1]: [item_sk#47] +Right keys [1]: [item_sk#53] Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Join condition: (rk#51 >= rk#52) + +(73) Project [codegen id : 48] +Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] + +(74) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] + +(75) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [max(web_sales#54), max(store_sales#55)] +Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] + +(76) Filter [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) + +(77) RowToColumnar +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] + +(78) CometTakeOrderedAndProject +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#47 ASC NULLS FIRST,d_date#48 ASC NULLS FIRST], output=[item_sk#47,d_date#48,web_sales#49,store_sales#50,web_cumulative#62,store_cumulative#63]), [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63], 100, 0, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] + +(79) CometColumnarToRow [codegen id : 49] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (84) ++- * CometColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +(80) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) +(81) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +(82) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(83) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(81) BroadcastExchange +(84) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index 15a87f66b6..898288d55b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -1,216 +1,242 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Window expressions are not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometSortMergeJoin + :- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 99 out of 196 eligible operators (50%). Final plan contains 44 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt index b3013059b0..91b14603ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt @@ -1,116 +1,131 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow +WholeStageCodegen (49) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + RowToColumnar + WholeStageCodegen (48) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow + Window [item_sk,d_date] + WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow + BroadcastExchange #7 + WholeStageCodegen (9) + Project [item_sk,sumws,rk] InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow + Window [ws_item_sk,d_date] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #8 + WholeStageCodegen (22) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (21) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #11 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (20) + Project [item_sk,sumss,rk] InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow + BroadcastExchange #13 + WholeStageCodegen (47) + Project [item_sk,web_sales,store_sales,rk] InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + Window [item_sk,d_date] + WholeStageCodegen (46) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt index f8a95a6db8..204830d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt @@ -1,80 +1,83 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) +* CometColumnarToRow (79) ++- CometTakeOrderedAndProject (78) + +- RowToColumnar (77) + +- * Filter (76) + +- * HashAggregate (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- Window (65) + : +- * CometColumnarToRow (64) + : +- CometSort (63) + : +- CometExchange (62) + : +- CometProject (61) + : +- CometFilter (60) + : +- CometSortMergeJoin (59) + : :- CometSort (29) + : : +- CometColumnarExchange (28) + : : +- * HashAggregate (27) + : : +- * CometColumnarToRow (26) + : : +- CometColumnarExchange (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (15) + : : : +- Window (14) + : : : +- * CometColumnarToRow (13) + : : : +- CometSort (12) + : : : +- CometColumnarExchange (11) + : : : +- * HashAggregate (10) + : : : +- * CometColumnarToRow (9) + : : : +- CometColumnarExchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * CometColumnarToRow (18) + : : +- CometSort (17) + : : +- ReusedExchange (16) + : +- CometSort (58) + : +- CometColumnarExchange (57) + : +- * HashAggregate (56) + : +- * CometColumnarToRow (55) + : +- CometColumnarExchange (54) + : +- * HashAggregate (53) + : +- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (44) + : : +- Window (43) + : : +- * CometColumnarToRow (42) + : : +- CometSort (41) + : : +- CometColumnarExchange (40) + : : +- * HashAggregate (39) + : : +- * CometColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * CometColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) + : : +- ReusedExchange (33) + : +- BroadcastExchange (50) + : +- * Project (49) + : +- Window (48) + : +- * CometColumnarToRow (47) + : +- CometSort (46) + : +- ReusedExchange (45) + +- BroadcastExchange (71) + +- * Project (70) + +- Window (69) + +- * CometColumnarToRow (68) + +- CometSort (67) + +- ReusedExchange (66) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -89,360 +92,372 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] +(3) CometColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] +(4) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#5, d_date#6] -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 2] +Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) CometHashAggregate +(7) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum#7] +Results [3]: [ws_item_sk#1, d_date#6, sum#8] + +(8) CometColumnarExchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(10) CometExchange +(9) CometColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometHashAggregate +(10) HashAggregate [codegen id : 3] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(11) CometColumnarExchange +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(12) CometSort +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +(13) CometColumnarToRow [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(14) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] +(15) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(16) ReusedExchange [Reuses operator id: 11] +Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] +(17) CometSort +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +(18) CometColumnarToRow [codegen id : 8] +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +(19) Window +Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] +(20) Project [codegen id : 9] +Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] +Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] +(21) BroadcastExchange +Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] +(22) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#16] Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Join condition: (rk#12 >= rk#15) + +(23) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] + +(24) HashAggregate [codegen id : 10] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [partial_sum(sumws#17)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(25) CometColumnarExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(26) CometColumnarToRow [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(27) HashAggregate [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [sum(sumws#17)] +Aggregate Attributes [1]: [sum(sumws#17)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] + +(28) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(29) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) +(31) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) + +(32) CometColumnarToRow [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] + +(33) ReusedExchange [Reuses operator id: 84] +Output [2]: [d_date_sk#28, d_date#29] -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] +(34) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight +(35) Project [codegen id : 13] +Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] +(36) HashAggregate [codegen id : 13] +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [ss_item_sk#24, d_date#29, sum#31] -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] +(37) CometColumnarExchange +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(38) CometColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] +(39) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] +Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(40) CometColumnarExchange +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(41) CometSort +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +(42) CometColumnarToRow [codegen id : 15] +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] +(43) Window +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] +(44) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] +Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(45) ReusedExchange [Reuses operator id: 40] +Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] +(46) CometSort +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(47) CometColumnarToRow [codegen id : 19] +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] +(48) Window +Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] +(49) Project [codegen id : 20] +Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] +Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] +(50) BroadcastExchange +Input [3]: [item_sk#39, sumss#40, rk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] +(51) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [item_sk#33] +Right keys [1]: [item_sk#39] Join type: Inner -Join condition: (rk#32 >= rk#35) +Join condition: (rk#35 >= rk#38) -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] +(52) Project [codegen id : 21] +Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +(53) HashAggregate [codegen id : 21] +Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [partial_sum(sumss#40)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(54) CometColumnarExchange +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +(55) CometColumnarToRow [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] +(56) HashAggregate [codegen id : 22] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [sum(sumss#40)] +Aggregate Attributes [1]: [sum(sumss#40)#45] +Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(57) CometColumnarExchange +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(58) CometSort +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter +(59) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#23] +Right output [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) +(60) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] +(61) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(62) CometExchange +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] +(63) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(64) CometColumnarToRow [codegen id : 23] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] +(65) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(66) ReusedExchange [Reuses operator id: 62] +Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] +(67) CometSort +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +(68) CometColumnarToRow [codegen id : 46] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] +(69) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] +(70) Project [codegen id : 47] +Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] +Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] +(71) BroadcastExchange +Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] +(72) BroadcastHashJoin [codegen id : 48] +Left keys [1]: [item_sk#47] +Right keys [1]: [item_sk#53] Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Join condition: (rk#51 >= rk#52) + +(73) Project [codegen id : 48] +Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] + +(74) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] + +(75) HashAggregate [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [max(web_sales#54), max(store_sales#55)] +Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] + +(76) Filter [codegen id : 48] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) + +(77) RowToColumnar +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] + +(78) CometTakeOrderedAndProject +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#47 ASC NULLS FIRST,d_date#48 ASC NULLS FIRST], output=[item_sk#47,d_date#48,web_sales#49,store_sales#50,web_cumulative#62,store_cumulative#63]), [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63], 100, 0, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] + +(79) CometColumnarToRow [codegen id : 49] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (84) ++- * CometColumnarToRow (83) + +- CometProject (82) + +- CometFilter (81) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +(80) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) +(81) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +(82) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(83) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(81) BroadcastExchange +(84) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt index 15a87f66b6..898288d55b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt @@ -1,216 +1,242 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Window expressions are not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometSortMergeJoin + :- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 99 out of 196 eligible operators (50%). Final plan contains 44 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index b3013059b0..91b14603ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -1,116 +1,131 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow +WholeStageCodegen (49) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + RowToColumnar + WholeStageCodegen (48) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow + Window [item_sk,d_date] + WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow + BroadcastExchange #7 + WholeStageCodegen (9) + Project [item_sk,sumws,rk] InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow + Window [ws_item_sk,d_date] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #8 + WholeStageCodegen (22) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (21) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #11 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (20) + Project [item_sk,sumss,rk] InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow + BroadcastExchange #13 + WholeStageCodegen (47) + Project [item_sk,web_sales,store_sales,rk] InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + Window [item_sk,d_date] + WholeStageCodegen (46) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt index d1daee98c1..eb446effff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt @@ -1,52 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (53) ++- CometTakeOrderedAndProject (52) + +- RowToColumnar (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * CometColumnarToRow (26) + : : +- CometSort (25) + : : +- CometColumnarExchange (24) + : : +- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- Window (38) + : +- * CometColumnarToRow (37) + : +- CometSort (36) + : +- CometColumnarExchange (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + +- BroadcastExchange (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -64,7 +69,10 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(4) CometColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#4, i_category#5] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -72,46 +80,39 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight +(8) BroadcastExchange +Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [cs_item_sk#7] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 4] +Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#14, cc_name#15] @@ -124,166 +125,188 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometBroadcastExchange +(16) CometColumnarToRow [codegen id : 3] Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight +(17) BroadcastExchange +Input [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) CometProject +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_call_center_sk#6] +Right keys [1]: [cc_call_center_sk#14] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -(19) CometHashAggregate +(20) HashAggregate [codegen id : 4] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] +Aggregate Attributes [1]: [sum#16] +Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] + +(21) CometColumnarExchange +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] +(23) HashAggregate [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] +Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(24) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) CometSort +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +(26) CometColumnarToRow [codegen id : 6] +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(27) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] +(28) Filter [codegen id : 7] +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +(29) Window +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(30) Filter [codegen id : 22] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(31) Project [codegen id : 22] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] +(32) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] +(33) CometColumnarToRow [codegen id : 12] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(34) HashAggregate [codegen id : 12] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] +Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +(35) CometColumnarExchange +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +(36) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +(37) CometColumnarToRow [codegen id : 13] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +(38) Window +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +(39) Project [codegen id : 14] +Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] +Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +(40) BroadcastExchange +Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] +Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +(42) Project [codegen id : 22] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +(43) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] +(44) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +(45) CometColumnarToRow [codegen id : 20] +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] +(46) Window +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(47) Project [codegen id : 21] +Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] +Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(48) BroadcastExchange +Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +(49) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] +Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(50) Project [codegen id : 22] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] + +(51) RowToColumnar +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] + +(52) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(53) CometColumnarToRow [codegen id : 23] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (57) ++- * CometColumnarToRow (56) + +- CometFilter (55) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(55) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index 7bee0e61e8..59b10afc93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -1,105 +1,122 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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 - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt index d655789fe6..3340fba52e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt @@ -1,71 +1,91 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] +WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + RowToColumnar + WholeStageCodegen (22) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (7) + Filter [d_year] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt index d1daee98c1..eb446effff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt @@ -1,52 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (53) ++- CometTakeOrderedAndProject (52) + +- RowToColumnar (51) + +- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * CometColumnarToRow (26) + : : +- CometSort (25) + : : +- CometColumnarExchange (24) + : : +- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (40) + : +- * Project (39) + : +- Window (38) + : +- * CometColumnarToRow (37) + : +- CometSort (36) + : +- CometColumnarExchange (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + +- BroadcastExchange (48) + +- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -64,7 +69,10 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(4) CometColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#4, i_category#5] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -72,46 +80,39 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight +(8) BroadcastExchange +Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [cs_item_sk#7] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 4] +Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 57] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#14, cc_name#15] @@ -124,166 +125,188 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometBroadcastExchange +(16) CometColumnarToRow [codegen id : 3] Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight +(17) BroadcastExchange +Input [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) CometProject +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_call_center_sk#6] +Right keys [1]: [cc_call_center_sk#14] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -(19) CometHashAggregate +(20) HashAggregate [codegen id : 4] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] +Aggregate Attributes [1]: [sum#16] +Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] + +(21) CometColumnarExchange +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(22) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] +(23) HashAggregate [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] +Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(24) CometColumnarExchange +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) CometSort +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +(26) CometColumnarToRow [codegen id : 6] +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(27) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] +(28) Filter [codegen id : 7] +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +(29) Window +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(30) Filter [codegen id : 22] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(31) Project [codegen id : 22] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] +(32) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] +(33) CometColumnarToRow [codegen id : 12] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(34) HashAggregate [codegen id : 12] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] +Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +(35) CometColumnarExchange +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +(36) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +(37) CometColumnarToRow [codegen id : 13] +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +(38) Window +Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +(39) Project [codegen id : 14] +Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] +Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +(40) BroadcastExchange +Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] +Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +(42) Project [codegen id : 22] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +(43) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] +(44) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +(45) CometColumnarToRow [codegen id : 20] +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] +(46) Window +Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(47) Project [codegen id : 21] +Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] +Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(48) BroadcastExchange +Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +(49) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] +Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(50) Project [codegen id : 22] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] + +(51) RowToColumnar +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] + +(52) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(53) CometColumnarToRow [codegen id : 23] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (57) ++- * CometColumnarToRow (56) + +- CometFilter (55) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(55) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(57) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt index 7bee0e61e8..59b10afc93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt @@ -1,105 +1,122 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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: Window expressions are not supported] - : +- CometColumnarToRow - : +- 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 - : : : +- CometColumnarToRow - : : : +- 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: Window expressions are not supported] - +- CometColumnarToRow - +- 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 - : : +- CometColumnarToRow - : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index d655789fe6..3340fba52e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -1,71 +1,91 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] +WholeStageCodegen (23) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + RowToColumnar + WholeStageCodegen (22) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (7) + Filter [d_year] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt index 33c3a5de2e..c094c4b82b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt @@ -1,88 +1,101 @@ == Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) +* CometColumnarToRow (97) ++- CometTakeOrderedAndProject (96) + +- CometHashAggregate (95) + +- CometColumnarExchange (94) + +- * HashAggregate (93) + +- Union (92) + :- * HashAggregate (77) + : +- * CometColumnarToRow (76) + : +- CometColumnarExchange (75) + : +- * HashAggregate (74) + : +- Union (73) + : :- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * CometColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (16) + : : +- * CometColumnarToRow (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) + : :- * HashAggregate (44) + : : +- * CometColumnarToRow (43) + : : +- CometColumnarExchange (42) + : : +- * HashAggregate (41) + : : +- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * CometColumnarToRow (30) + : : : : +- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (31) + : : +- BroadcastExchange (38) + : : +- * CometColumnarToRow (37) + : : +- CometProject (36) + : : +- CometFilter (35) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) + : +- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- * HashAggregate (69) + : +- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- Union (58) + : : : :- * CometColumnarToRow (48) + : : : : +- CometProject (47) + : : : : +- CometFilter (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) + : : : +- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildLeft (56) + : : : :- BroadcastExchange (51) + : : : : +- * CometColumnarToRow (50) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (49) + : : : +- * CometColumnarToRow (55) + : : : +- CometProject (54) + : : : +- CometFilter (53) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (62) + :- * HashAggregate (84) + : +- * CometColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- * HashAggregate (81) + : +- * HashAggregate (80) + : +- * CometColumnarToRow (79) + : +- ReusedExchange (78) + +- * HashAggregate (91) + +- * CometColumnarToRow (90) + +- CometColumnarExchange (89) + +- * HashAggregate (88) + +- * HashAggregate (87) + +- * CometColumnarToRow (86) + +- ReusedExchange (85) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -121,404 +134,465 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] +(8) CometColumnarToRow [codegen id : 3] +Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] +(9) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#22] -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None -(13) CometProject +(11) Project [codegen id : 3] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#23, s_store_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) +(13) CometFilter +Input [2]: [s_store_sk#23, s_store_id#24] +Condition : isnotnull(s_store_sk#23) + +(14) CometProject +Input [2]: [s_store_sk#23, s_store_id#24] +Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] +(15) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#23, s_store_id#25] -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] +(16) BroadcastExchange +Input [2]: [s_store_sk#23, s_store_id#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight +(17) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [store_sk#6] +Right keys [1]: [s_store_sk#23] +Join type: Inner +Join condition: None -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] +(18) Project [codegen id : 3] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] +(19) HashAggregate [codegen id : 3] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(20) CometColumnarExchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] +(21) CometColumnarToRow [codegen id : 4] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(22) HashAggregate [codegen id : 4] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] (23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct (24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) (25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct (27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) (28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] (29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] +Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] + +(30) CometColumnarToRow [codegen id : 7] +Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] +(31) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [date_sk#49] +Right keys [1]: [d_date_sk#64] +Join type: Inner +Join condition: None -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +(33) Project [codegen id : 7] +Output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] +Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +(35) CometFilter +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) + +(36) CometProject +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#67], [cp_catalog_page_sk#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#66, 16, true, false, true) AS cp_catalog_page_id#67] + +(37) CometColumnarToRow [codegen id : 6] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] + +(38) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(39) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [page_sk#48] +Right keys [1]: [cp_catalog_page_sk#65] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 7] +Output [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] +Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#67] + +(41) HashAggregate [codegen id : 7] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] +Keys [1]: [cp_catalog_page_id#67] +Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] +Results [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] + +(42) CometColumnarExchange +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(cp_catalog_page_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(43) CometColumnarToRow [codegen id : 8] +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] + +(44) HashAggregate [codegen id : 8] +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] +Keys [1]: [cp_catalog_page_id#67] +Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#76, sum(UnscaledValue(return_amt#52))#77, sum(UnscaledValue(profit#51))#78, sum(UnscaledValue(net_loss#53))#79] +Results [5]: [catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#67) AS id#81, MakeDecimal(sum(UnscaledValue(sales_price#50))#76,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(return_amt#52))#77,17,2) AS returns#83, (MakeDecimal(sum(UnscaledValue(profit#51))#78,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#79,17,2)) AS profit#84] + +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(ws_sold_date_sk#88), dynamicpruningexpression(ws_sold_date_sk#88 IN dynamicpruning#89)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) +(46) CometFilter +Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Condition : isnotnull(ws_web_site_sk#85) + +(47) CometProject +Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Arguments: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95], [ws_web_site_sk#85 AS wsr_web_site_sk#90, ws_sold_date_sk#88 AS date_sk#91, ws_ext_sales_price#86 AS sales_price#92, ws_net_profit#87 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] +(48) CometColumnarToRow [codegen id : 9] +Input [6]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(wr_returned_date_sk#100), dynamicpruningexpression(wr_returned_date_sk#100 IN dynamicpruning#89)] ReadSchema: struct -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +(50) CometColumnarToRow [codegen id : 10] +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] + +(51) BroadcastExchange +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) +(53) CometFilter +Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +Condition : ((isnotnull(ws_item_sk#101) AND isnotnull(ws_order_number#103)) AND isnotnull(ws_web_site_sk#102)) -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +(54) CometProject +Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +Arguments: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103], [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft +(55) CometColumnarToRow +Input [3]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] +(56) BroadcastHashJoin [codegen id : 11] +Left keys [2]: [wr_item_sk#96, wr_order_number#97] +Right keys [2]: [ws_item_sk#101, ws_order_number#103] +Join type: Inner +Join condition: None -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] +(57) Project [codegen id : 11] +Output [6]: [ws_web_site_sk#102 AS wsr_web_site_sk#105, wr_returned_date_sk#100 AS date_sk#106, 0.00 AS sales_price#107, 0.00 AS profit#108, wr_return_amt#98 AS return_amt#109, wr_net_loss#99 AS net_loss#110] +Input [8]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100, ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] +(58) Union -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#111] -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [date_sk#91] +Right keys [1]: [d_date_sk#111] +Join type: Inner +Join condition: None -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] +(61) Project [codegen id : 14] +Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] +Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#111] + +(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#124, returns#125, profit#126] +(63) CometFilter +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) + +(64) CometProject +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: [web_site_sk#112, web_site_id#114], [web_site_sk#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#113, 16, true, false, true) AS web_site_id#114] + +(65) CometColumnarToRow [codegen id : 13] +Input [2]: [web_site_sk#112, web_site_id#114] + +(66) BroadcastExchange +Input [2]: [web_site_sk#112, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [wsr_web_site_sk#90] +Right keys [1]: [web_site_sk#112] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#112, web_site_id#114] + +(69) HashAggregate [codegen id : 14] +Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] + +(70) CometColumnarExchange +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(71) CometColumnarToRow [codegen id : 15] +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] + +(72) HashAggregate [codegen id : 15] +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#123, sum(UnscaledValue(return_amt#94))#124, sum(UnscaledValue(profit#93))#125, sum(UnscaledValue(net_loss#95))#126] +Results [5]: [web channel AS channel#127, concat(web_site, web_site_id#114) AS id#128, MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS returns#130, (MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2)) AS profit#131] + +(73) Union + +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] +Keys [2]: [channel#38, id#39] +Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(75) CometColumnarExchange +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(76) CometColumnarToRow [codegen id : 17] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(77) HashAggregate [codegen id : 17] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [5]: [channel#38, id#39, cast(sum(sales#40)#144 as decimal(37,2)) AS sales#147, cast(sum(returns#41)#145 as decimal(37,2)) AS returns#148, cast(sum(profit#42)#146 as decimal(38,2)) AS profit#149] + +(78) ReusedExchange [Reuses operator id: 75] +Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(79) CometColumnarToRow [codegen id : 34] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(80) HashAggregate [codegen id : 34] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [4]: [channel#38, sum(sales#40)#144 AS sales#150, sum(returns#41)#145 AS returns#151, sum(profit#42)#146 AS profit#152] + +(81) HashAggregate [codegen id : 34] +Input [4]: [channel#38, sales#150, returns#151, profit#152] +Keys [1]: [channel#38] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] + +(82) CometColumnarExchange +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(83) CometColumnarToRow [codegen id : 35] +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] + +(84) HashAggregate [codegen id : 35] +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Keys [1]: [channel#38] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [channel#38, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] + +(85) ReusedExchange [Reuses operator id: 75] +Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(86) CometColumnarToRow [codegen id : 52] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(87) HashAggregate [codegen id : 52] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [3]: [sum(sales#40)#144 AS sales#172, sum(returns#41)#145 AS returns#173, sum(profit#42)#146 AS profit#174] + +(88) HashAggregate [codegen id : 52] +Input [3]: [sales#172, returns#173, profit#174] Keys: [] -Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Functions [3]: [partial_sum(sales#172), partial_sum(returns#173), partial_sum(profit#174)] +Aggregate Attributes [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Results [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] + +(89) CometColumnarExchange +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(77) CometExchange -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(90) CometColumnarToRow [codegen id : 53] +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] -(78) CometHashAggregate -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +(91) HashAggregate [codegen id : 53] +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] Keys: [] -Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Functions [3]: [sum(sales#172), sum(returns#173), sum(profit#174)] +Aggregate Attributes [3]: [sum(sales#172)#187, sum(returns#173)#188, sum(profit#174)#189] +Results [5]: [null AS channel#190, null AS id#191, sum(sales#172)#187 AS sum(sales)#192, sum(returns#173)#188 AS sum(returns)#193, sum(profit#174)#189 AS sum(profit)#194] -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] -Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] +(92) Union -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(93) HashAggregate [codegen id : 54] +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -(81) CometExchange -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(94) CometColumnarExchange +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Arguments: hashpartitioning(channel#38, id#39, sales#147, returns#148, profit#149, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(95) CometHashAggregate +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] Functions: [] -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] +(96) CometTakeOrderedAndProject +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#147,returns#148,profit#149]), [channel#38, id#39, sales#147, returns#148, profit#149], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#147, returns#148, profit#149] -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(97) CometColumnarToRow [codegen id : 55] +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) +BroadcastExchange (102) ++- * CometColumnarToRow (101) + +- CometProject (100) + +- CometFilter (99) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (98) -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#195] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(99) CometFilter +Input [2]: [d_date_sk#22, d_date#195] +Condition : (((isnotnull(d_date#195) AND (d_date#195 >= 1998-08-04)) AND (d_date#195 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] +(100) CometProject +Input [2]: [d_date_sk#22, d_date#195] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(89) BroadcastExchange +(102) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#88 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 49 Hosting Expression = wr_returned_date_sk#100 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..983467d51e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -1,269 +1,316 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- 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 - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- 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 - : : : : +- 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.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- 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.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- 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 - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- 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 - : : : : +- 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.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- 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 + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- 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 - : : : +- 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.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- 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.web_site + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- 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 + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometUnion + : : : :- 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 + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 143 out of 263 eligible operators (54%). Final plan contains 51 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt index c0b236a57a..3f6eeb8d6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt @@ -1,99 +1,157 @@ -WholeStageCodegen (1) +WholeStageCodegen (55) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (54) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (17) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (16) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (15) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 + WholeStageCodegen (14) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (11) + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (35) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #11 + WholeStageCodegen (34) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #12 + WholeStageCodegen (52) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index 33c3a5de2e..c094c4b82b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -1,88 +1,101 @@ == Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) +* CometColumnarToRow (97) ++- CometTakeOrderedAndProject (96) + +- CometHashAggregate (95) + +- CometColumnarExchange (94) + +- * HashAggregate (93) + +- Union (92) + :- * HashAggregate (77) + : +- * CometColumnarToRow (76) + : +- CometColumnarExchange (75) + : +- * HashAggregate (74) + : +- Union (73) + : :- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * CometColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (16) + : : +- * CometColumnarToRow (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) + : :- * HashAggregate (44) + : : +- * CometColumnarToRow (43) + : : +- CometColumnarExchange (42) + : : +- * HashAggregate (41) + : : +- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * CometColumnarToRow (30) + : : : : +- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (31) + : : +- BroadcastExchange (38) + : : +- * CometColumnarToRow (37) + : : +- CometProject (36) + : : +- CometFilter (35) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) + : +- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- * HashAggregate (69) + : +- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- Union (58) + : : : :- * CometColumnarToRow (48) + : : : : +- CometProject (47) + : : : : +- CometFilter (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) + : : : +- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildLeft (56) + : : : :- BroadcastExchange (51) + : : : : +- * CometColumnarToRow (50) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (49) + : : : +- * CometColumnarToRow (55) + : : : +- CometProject (54) + : : : +- CometFilter (53) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (62) + :- * HashAggregate (84) + : +- * CometColumnarToRow (83) + : +- CometColumnarExchange (82) + : +- * HashAggregate (81) + : +- * HashAggregate (80) + : +- * CometColumnarToRow (79) + : +- ReusedExchange (78) + +- * HashAggregate (91) + +- * CometColumnarToRow (90) + +- CometColumnarExchange (89) + +- * HashAggregate (88) + +- * HashAggregate (87) + +- * CometColumnarToRow (86) + +- ReusedExchange (85) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -121,404 +134,465 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] +(8) CometColumnarToRow [codegen id : 3] +Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] +(9) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#22] -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None -(13) CometProject +(11) Project [codegen id : 3] +Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#23, s_store_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) +(13) CometFilter +Input [2]: [s_store_sk#23, s_store_id#24] +Condition : isnotnull(s_store_sk#23) + +(14) CometProject +Input [2]: [s_store_sk#23, s_store_id#24] +Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] +(15) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#23, s_store_id#25] -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] +(16) BroadcastExchange +Input [2]: [s_store_sk#23, s_store_id#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight +(17) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [store_sk#6] +Right keys [1]: [s_store_sk#23] +Join type: Inner +Join condition: None -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] +(18) Project [codegen id : 3] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] +(19) HashAggregate [codegen id : 3] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(20) CometColumnarExchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] +(21) CometColumnarToRow [codegen id : 4] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(22) HashAggregate [codegen id : 4] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] (23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct (24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) (25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct (27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) (28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] (29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] +Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] + +(30) CometColumnarToRow [codegen id : 7] +Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] +(31) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [date_sk#49] +Right keys [1]: [d_date_sk#64] +Join type: Inner +Join condition: None -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +(33) Project [codegen id : 7] +Output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] +Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +(35) CometFilter +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) + +(36) CometProject +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#67], [cp_catalog_page_sk#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#66, 16, true, false, true) AS cp_catalog_page_id#67] + +(37) CometColumnarToRow [codegen id : 6] +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] + +(38) BroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(39) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [page_sk#48] +Right keys [1]: [cp_catalog_page_sk#65] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 7] +Output [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] +Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#67] + +(41) HashAggregate [codegen id : 7] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] +Keys [1]: [cp_catalog_page_id#67] +Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] +Results [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] + +(42) CometColumnarExchange +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(cp_catalog_page_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(43) CometColumnarToRow [codegen id : 8] +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] + +(44) HashAggregate [codegen id : 8] +Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] +Keys [1]: [cp_catalog_page_id#67] +Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#76, sum(UnscaledValue(return_amt#52))#77, sum(UnscaledValue(profit#51))#78, sum(UnscaledValue(net_loss#53))#79] +Results [5]: [catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#67) AS id#81, MakeDecimal(sum(UnscaledValue(sales_price#50))#76,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(return_amt#52))#77,17,2) AS returns#83, (MakeDecimal(sum(UnscaledValue(profit#51))#78,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#79,17,2)) AS profit#84] + +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(ws_sold_date_sk#88), dynamicpruningexpression(ws_sold_date_sk#88 IN dynamicpruning#89)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) +(46) CometFilter +Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Condition : isnotnull(ws_web_site_sk#85) + +(47) CometProject +Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +Arguments: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95], [ws_web_site_sk#85 AS wsr_web_site_sk#90, ws_sold_date_sk#88 AS date_sk#91, ws_ext_sales_price#86 AS sales_price#92, ws_net_profit#87 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] +(48) CometColumnarToRow [codegen id : 9] +Input [6]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] +PartitionFilters: [isnotnull(wr_returned_date_sk#100), dynamicpruningexpression(wr_returned_date_sk#100 IN dynamicpruning#89)] ReadSchema: struct -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +(50) CometColumnarToRow [codegen id : 10] +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] + +(51) BroadcastExchange +Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) +(53) CometFilter +Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +Condition : ((isnotnull(ws_item_sk#101) AND isnotnull(ws_order_number#103)) AND isnotnull(ws_web_site_sk#102)) -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +(54) CometProject +Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +Arguments: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103], [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft +(55) CometColumnarToRow +Input [3]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] +(56) BroadcastHashJoin [codegen id : 11] +Left keys [2]: [wr_item_sk#96, wr_order_number#97] +Right keys [2]: [ws_item_sk#101, ws_order_number#103] +Join type: Inner +Join condition: None -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] +(57) Project [codegen id : 11] +Output [6]: [ws_web_site_sk#102 AS wsr_web_site_sk#105, wr_returned_date_sk#100 AS date_sk#106, 0.00 AS sales_price#107, 0.00 AS profit#108, wr_return_amt#98 AS return_amt#109, wr_net_loss#99 AS net_loss#110] +Input [8]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100, ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] +(58) Union -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 102] +Output [1]: [d_date_sk#111] -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [date_sk#91] +Right keys [1]: [d_date_sk#111] +Join type: Inner +Join condition: None -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] +(61) Project [codegen id : 14] +Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] +Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#111] + +(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#124, returns#125, profit#126] +(63) CometFilter +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) + +(64) CometProject +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: [web_site_sk#112, web_site_id#114], [web_site_sk#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#113, 16, true, false, true) AS web_site_id#114] + +(65) CometColumnarToRow [codegen id : 13] +Input [2]: [web_site_sk#112, web_site_id#114] + +(66) BroadcastExchange +Input [2]: [web_site_sk#112, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [wsr_web_site_sk#90] +Right keys [1]: [web_site_sk#112] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#112, web_site_id#114] + +(69) HashAggregate [codegen id : 14] +Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] + +(70) CometColumnarExchange +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(71) CometColumnarToRow [codegen id : 15] +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] + +(72) HashAggregate [codegen id : 15] +Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#123, sum(UnscaledValue(return_amt#94))#124, sum(UnscaledValue(profit#93))#125, sum(UnscaledValue(net_loss#95))#126] +Results [5]: [web channel AS channel#127, concat(web_site, web_site_id#114) AS id#128, MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS returns#130, (MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2)) AS profit#131] + +(73) Union + +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] +Keys [2]: [channel#38, id#39] +Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(75) CometColumnarExchange +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(76) CometColumnarToRow [codegen id : 17] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(77) HashAggregate [codegen id : 17] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [5]: [channel#38, id#39, cast(sum(sales#40)#144 as decimal(37,2)) AS sales#147, cast(sum(returns#41)#145 as decimal(37,2)) AS returns#148, cast(sum(profit#42)#146 as decimal(38,2)) AS profit#149] + +(78) ReusedExchange [Reuses operator id: 75] +Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(79) CometColumnarToRow [codegen id : 34] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(80) HashAggregate [codegen id : 34] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [4]: [channel#38, sum(sales#40)#144 AS sales#150, sum(returns#41)#145 AS returns#151, sum(profit#42)#146 AS profit#152] + +(81) HashAggregate [codegen id : 34] +Input [4]: [channel#38, sales#150, returns#151, profit#152] +Keys [1]: [channel#38] +Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] +Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Results [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] + +(82) CometColumnarExchange +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(83) CometColumnarToRow [codegen id : 35] +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] + +(84) HashAggregate [codegen id : 35] +Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Keys [1]: [channel#38] +Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] +Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] +Results [5]: [channel#38, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] + +(85) ReusedExchange [Reuses operator id: 75] +Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(86) CometColumnarToRow [codegen id : 52] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(87) HashAggregate [codegen id : 52] +Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] +Results [3]: [sum(sales#40)#144 AS sales#172, sum(returns#41)#145 AS returns#173, sum(profit#42)#146 AS profit#174] + +(88) HashAggregate [codegen id : 52] +Input [3]: [sales#172, returns#173, profit#174] Keys: [] -Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] +Functions [3]: [partial_sum(sales#172), partial_sum(returns#173), partial_sum(profit#174)] +Aggregate Attributes [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Results [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] + +(89) CometColumnarExchange +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(77) CometExchange -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(90) CometColumnarToRow [codegen id : 53] +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] -(78) CometHashAggregate -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +(91) HashAggregate [codegen id : 53] +Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] Keys: [] -Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] +Functions [3]: [sum(sales#172), sum(returns#173), sum(profit#174)] +Aggregate Attributes [3]: [sum(sales#172)#187, sum(returns#173)#188, sum(profit#174)#189] +Results [5]: [null AS channel#190, null AS id#191, sum(sales#172)#187 AS sum(sales)#192, sum(returns#173)#188 AS sum(returns)#193, sum(profit#174)#189 AS sum(profit)#194] -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] -Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] +(92) Union -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(93) HashAggregate [codegen id : 54] +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -(81) CometExchange -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(94) CometColumnarExchange +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Arguments: hashpartitioning(channel#38, id#39, sales#147, returns#148, profit#149, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(95) CometHashAggregate +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] Functions: [] -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] +(96) CometTakeOrderedAndProject +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#147,returns#148,profit#149]), [channel#38, id#39, sales#147, returns#148, profit#149], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#147, returns#148, profit#149] -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +(97) CometColumnarToRow [codegen id : 55] +Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) +BroadcastExchange (102) ++- * CometColumnarToRow (101) + +- CometProject (100) + +- CometFilter (99) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (98) -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#195] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(99) CometFilter +Input [2]: [d_date_sk#22, d_date#195] +Condition : (((isnotnull(d_date#195) AND (d_date#195 >= 1998-08-04)) AND (d_date#195 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] +(100) CometProject +Input [2]: [d_date_sk#22, d_date#195] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(89) BroadcastExchange +(102) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#88 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 49 Hosting Expression = wr_returned_date_sk#100 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt index 35b86dbff5..983467d51e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt @@ -1,269 +1,316 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- 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 - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- 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 - : : : : +- 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.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- 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.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- 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 - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- 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 - : : : : +- 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.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- 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 + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- 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 - : : : +- 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.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- 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.web_site + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometUnion + : : : : :- 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 + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometUnion + : : : :- 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 + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 143 out of 263 eligible operators (54%). Final plan contains 51 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index c0b236a57a..3f6eeb8d6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -1,99 +1,157 @@ -WholeStageCodegen (1) +WholeStageCodegen (55) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (54) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (17) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (16) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (15) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #8 + WholeStageCodegen (14) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (11) + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (35) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #11 + WholeStageCodegen (34) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #12 + WholeStageCodegen (52) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt index 73dd1d4955..c984427acb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt @@ -1,44 +1,48 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- RowToColumnar (42) + +- * Filter (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * CometColumnarToRow (4) + : : : : +- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (8) + : : : +- * CometColumnarToRow (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (5) + : : +- BroadcastExchange (14) + : : +- * CometColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : +- ReusedExchange (17) + +- BroadcastExchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * CometColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) + +- BroadcastExchange (31) + +- * CometColumnarToRow (30) + +- CometFilter (29) + +- CometHashAggregate (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -56,31 +60,38 @@ Condition : isnotnull(ca_address_sk#1) Input [2]: [ca_address_sk#1, ca_state#2] Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(4) CometColumnarToRow [codegen id : 6] +Input [2]: [ca_address_sk#1, ca_state#3] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [2]: [c_customer_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [2]: [c_customer_sk#4, c_current_addr_sk#5] Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#4, c_current_addr_sk#5] + +(8) BroadcastExchange Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ca_address_sk#1] +Right keys [1]: [c_current_addr_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 6] +Output [2]: [ca_state#3, c_customer_sk#4] Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -88,219 +99,224 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(10) CometFilter +(12) CometFilter Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) -(11) CometBroadcastExchange +(13) CometColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +(14) BroadcastExchange +Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) +(15) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#4] +Right keys [1]: [ss_customer_sk#7] +Join type: Inner +Join condition: None -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] +(16) Project [codegen id : 6] +Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] +(17) ReusedExchange [Reuses operator id: 49] +Output [1]: [d_date_sk#10] -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None -(19) CometProject +(19) Project [codegen id : 6] +Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] (20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) +Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) + +(22) CometColumnarToRow [codegen id : 5] +Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) +(24) CometFilter +Input [2]: [i_current_price#14, i_category#15] +Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] +(25) CometProject +Input [2]: [i_current_price#14, i_category#15] +Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +(26) CometHashAggregate +Input [2]: [i_current_price#14, i_category#16] +Keys [1]: [i_category#16] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(27) CometExchange +Input [3]: [i_category#16, sum#17, count#18] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] +(28) CometHashAggregate +Input [3]: [i_category#16, sum#17, count#18] +Keys [1]: [i_category#16] +Functions [1]: [avg(UnscaledValue(i_current_price#14))] -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) +(29) CometFilter +Input [2]: [avg(i_current_price)#19, i_category#16] +Condition : isnotnull(avg(i_current_price)#19) -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] +(30) CometColumnarToRow [codegen id : 4] +Input [2]: [avg(i_current_price)#19, i_category#16] -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight +(31) BroadcastExchange +Input [2]: [avg(i_current_price)#19, i_category#16] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)] +Right keys [1]: [i_category#16] +Join type: Inner +Join condition: (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)) -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] +(33) Project [codegen id : 5] +Output [1]: [i_item_sk#11] +Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight +(34) BroadcastExchange +Input [1]: [i_item_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#11] +Join type: Inner +Join condition: None -(35) CometHashAggregate +(36) Project [codegen id : 6] +Output [1]: [ca_state#3] +Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] + +(37) HashAggregate [codegen id : 6] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#20] +Results [2]: [ca_state#3, count#21] + +(38) CometColumnarExchange +Input [2]: [ca_state#3, count#21] +Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(39) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_state#3, count#21] -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] +(40) HashAggregate [codegen id : 7] +Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#22] +Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] -(38) CometFilter -Input [3]: [state#24, cnt#25, ca_state#3] -Condition : (cnt#25 >= 10) +(41) Filter [codegen id : 7] +Input [3]: [state#23, cnt#24, ca_state#3] +Condition : (cnt#24 >= 10) -(39) CometTakeOrderedAndProject -Input [3]: [state#24, cnt#25, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] +(42) RowToColumnar +Input [3]: [state#23, cnt#24, ca_state#3] -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] +(43) CometTakeOrderedAndProject +Input [3]: [state#23, cnt#24, ca_state#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#24 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#23,cnt#24]), [state#23, cnt#24], 100, 0, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, cnt#24] + +(44) CometColumnarToRow [codegen id : 8] +Input [2]: [state#23, cnt#24] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (49) ++- * CometColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) +(46) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#25] +Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] +(47) CometProject +Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(45) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#26, [id=#27] +* CometColumnarToRow (56) ++- CometHashAggregate (55) + +- CometExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#28, d_year#29, d_moy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(51) CometFilter +Input [3]: [d_month_seq#28, d_year#29, d_moy#30] +Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(52) CometProject +Input [3]: [d_month_seq#28, d_year#29, d_moy#30] +Arguments: [d_month_seq#28], [d_month_seq#28] -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(53) CometHashAggregate +Input [1]: [d_month_seq#28] +Keys [1]: [d_month_seq#28] Functions: [] -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(54) CometExchange +Input [1]: [d_month_seq#28] +Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(55) CometHashAggregate +Input [1]: [d_month_seq#28] +Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] +(56) CometColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..3abe831786 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -1,57 +1,72 @@ CometColumnarToRow +- 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 - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- 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 + +- RowToColumnar + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 58 eligible operators (62%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt index cbaf71ab0d..81b3b20040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt @@ -1,61 +1,81 @@ -WholeStageCodegen (1) +WholeStageCodegen (8) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (7) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + WholeStageCodegen (6) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt index 73dd1d4955..c984427acb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt @@ -1,44 +1,48 @@ == Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- RowToColumnar (42) + +- * Filter (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * CometColumnarToRow (4) + : : : : +- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (8) + : : : +- * CometColumnarToRow (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (5) + : : +- BroadcastExchange (14) + : : +- * CometColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : +- ReusedExchange (17) + +- BroadcastExchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * CometColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) + +- BroadcastExchange (31) + +- * CometColumnarToRow (30) + +- CometFilter (29) + +- CometHashAggregate (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -56,31 +60,38 @@ Condition : isnotnull(ca_address_sk#1) Input [2]: [ca_address_sk#1, ca_state#2] Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(4) CometColumnarToRow [codegen id : 6] +Input [2]: [ca_address_sk#1, ca_state#3] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [2]: [c_customer_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [2]: [c_customer_sk#4, c_current_addr_sk#5] Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#4, c_current_addr_sk#5] + +(8) BroadcastExchange Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ca_address_sk#1] +Right keys [1]: [c_current_addr_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 6] +Output [2]: [ca_state#3, c_customer_sk#4] Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -88,219 +99,224 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(10) CometFilter +(12) CometFilter Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) -(11) CometBroadcastExchange +(13) CometColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct +(14) BroadcastExchange +Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) +(15) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#4] +Right keys [1]: [ss_customer_sk#7] +Join type: Inner +Join condition: None -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] +(16) Project [codegen id : 6] +Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] +(17) ReusedExchange [Reuses operator id: 49] +Output [1]: [d_date_sk#10] -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None -(19) CometProject +(19) Project [codegen id : 6] +Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] (20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) +Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) + +(22) CometColumnarToRow [codegen id : 5] +Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) +(24) CometFilter +Input [2]: [i_current_price#14, i_category#15] +Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] +(25) CometProject +Input [2]: [i_current_price#14, i_category#15] +Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +(26) CometHashAggregate +Input [2]: [i_current_price#14, i_category#16] +Keys [1]: [i_category#16] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(27) CometExchange +Input [3]: [i_category#16, sum#17, count#18] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] +(28) CometHashAggregate +Input [3]: [i_category#16, sum#17, count#18] +Keys [1]: [i_category#16] +Functions [1]: [avg(UnscaledValue(i_current_price#14))] -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) +(29) CometFilter +Input [2]: [avg(i_current_price)#19, i_category#16] +Condition : isnotnull(avg(i_current_price)#19) -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] +(30) CometColumnarToRow [codegen id : 4] +Input [2]: [avg(i_current_price)#19, i_category#16] -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight +(31) BroadcastExchange +Input [2]: [avg(i_current_price)#19, i_category#16] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)] +Right keys [1]: [i_category#16] +Join type: Inner +Join condition: (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)) -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] +(33) Project [codegen id : 5] +Output [1]: [i_item_sk#11] +Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight +(34) BroadcastExchange +Input [1]: [i_item_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#11] +Join type: Inner +Join condition: None -(35) CometHashAggregate +(36) Project [codegen id : 6] +Output [1]: [ca_state#3] +Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] + +(37) HashAggregate [codegen id : 6] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#20] +Results [2]: [ca_state#3, count#21] + +(38) CometColumnarExchange +Input [2]: [ca_state#3, count#21] +Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(39) CometColumnarToRow [codegen id : 7] +Input [2]: [ca_state#3, count#21] -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] +(40) HashAggregate [codegen id : 7] +Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#22] +Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] -(38) CometFilter -Input [3]: [state#24, cnt#25, ca_state#3] -Condition : (cnt#25 >= 10) +(41) Filter [codegen id : 7] +Input [3]: [state#23, cnt#24, ca_state#3] +Condition : (cnt#24 >= 10) -(39) CometTakeOrderedAndProject -Input [3]: [state#24, cnt#25, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] +(42) RowToColumnar +Input [3]: [state#23, cnt#24, ca_state#3] -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] +(43) CometTakeOrderedAndProject +Input [3]: [state#23, cnt#24, ca_state#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#24 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#23,cnt#24]), [state#23, cnt#24], 100, 0, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, cnt#24] + +(44) CometColumnarToRow [codegen id : 8] +Input [2]: [state#23, cnt#24] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (49) ++- * CometColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) +(46) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#25] +Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] +(47) CometProject +Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(45) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#26, [id=#27] +* CometColumnarToRow (56) ++- CometHashAggregate (55) + +- CometExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#28, d_year#29, d_moy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +(51) CometFilter +Input [3]: [d_month_seq#28, d_year#29, d_moy#30] +Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +(52) CometProject +Input [3]: [d_month_seq#28, d_year#29, d_moy#30] +Arguments: [d_month_seq#28], [d_month_seq#28] -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(53) CometHashAggregate +Input [1]: [d_month_seq#28] +Keys [1]: [d_month_seq#28] Functions: [] -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(54) CometExchange +Input [1]: [d_month_seq#28] +Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +(55) CometHashAggregate +Input [1]: [d_month_seq#28] +Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] +(56) CometColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt index f5b69fc6cf..3abe831786 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt @@ -1,57 +1,72 @@ CometColumnarToRow +- 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 - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- 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 + +- RowToColumnar + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 58 eligible operators (62%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index cbaf71ab0d..81b3b20040 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -1,61 +1,81 @@ -WholeStageCodegen (1) +WholeStageCodegen (8) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_sk] - CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] - CometProject [ca_state,ss_item_sk,ss_sold_date_sk] - CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) + RowToColumnar + WholeStageCodegen (7) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + WholeStageCodegen (6) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt index 077daeca3a..753820f88a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt @@ -1,180 +1,191 @@ == Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) +* CometColumnarToRow (187) ++- CometSort (186) + +- CometExchange (185) + +- CometProject (184) + +- CometSortMergeJoin (183) + :- CometSort (113) + : +- CometColumnarExchange (112) + : +- * HashAggregate (111) + : +- * HashAggregate (110) + : +- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * Project (102) + : : +- * BroadcastHashJoin Inner BuildRight (101) + : : :- * Project (99) + : : : +- * BroadcastHashJoin Inner BuildRight (98) + : : : :- * Project (93) + : : : : +- * BroadcastHashJoin Inner BuildRight (92) + : : : : :- * Project (90) + : : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : : :- * Project (83) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (82) + : : : : : : :- * Project (80) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : : : : :- * Project (74) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : : : : : : :- * Project (68) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (67) + : : : : : : : : : :- * Project (65) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) + : : : : : : : : : : :- * Project (58) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : : :- * Project (49) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : : : : : : : : :- * Project (43) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (33) + : : : : : : : : : : : : : : : : +- CometProject (32) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) + : : : : : : : : : : : : : : : : :- CometSort (12) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) + : : : : : : : : : : : : : : : : : +- * Project (10) + : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) + : : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometProject (7) + : : : : : : : : : : : : : : : : : +- CometFilter (6) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : : : : : : : : : : : : +- CometSort (30) + : : : : : : : : : : : : : : : : +- CometProject (29) + : : : : : : : : : : : : : : : : +- CometFilter (28) + : : : : : : : : : : : : : : : : +- CometHashAggregate (27) + : : : : : : : : : : : : : : : : +- CometExchange (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometProject (24) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) + : : : : : : : : : : : : : : : : :- CometSort (17) + : : : : : : : : : : : : : : : : : +- CometExchange (16) + : : : : : : : : : : : : : : : : : +- CometProject (15) + : : : : : : : : : : : : : : : : : +- CometFilter (14) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : : : : : : : : : : : +- CometSort (22) + : : : : : : : : : : : : : : : : +- CometExchange (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (41) + : : : : : : : : : : : : : : +- * CometColumnarToRow (40) + : : : : : : : : : : : : : : +- CometProject (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (47) + : : : : : : : : : : : : : +- * CometColumnarToRow (46) + : : : : : : : : : : : : : +- CometFilter (45) + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (44) + : : : : : : : : : : : : +- BroadcastExchange (53) + : : : : : : : : : : : : +- * CometColumnarToRow (52) + : : : : : : : : : : : : +- CometFilter (51) + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) + : : : : : : : : : : : +- ReusedExchange (56) + : : : : : : : : : : +- BroadcastExchange (63) + : : : : : : : : : : +- * CometColumnarToRow (62) + : : : : : : : : : : +- CometProject (61) + : : : : : : : : : : +- CometFilter (60) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (59) + : : : : : : : : : +- ReusedExchange (66) + : : : : : : : : +- BroadcastExchange (72) + : : : : : : : : +- * CometColumnarToRow (71) + : : : : : : : : +- CometFilter (70) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (69) + : : : : : : : +- BroadcastExchange (78) + : : : : : : : +- * CometColumnarToRow (77) + : : : : : : : +- CometFilter (76) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (75) + : : : : : : +- ReusedExchange (81) + : : : : : +- BroadcastExchange (88) + : : : : : +- * CometColumnarToRow (87) + : : : : : +- CometProject (86) + : : : : : +- CometFilter (85) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (84) + : : : : +- ReusedExchange (91) + : : : +- BroadcastExchange (97) + : : : +- * CometColumnarToRow (96) + : : : +- CometFilter (95) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (94) + : : +- ReusedExchange (100) + : +- BroadcastExchange (107) + : +- * CometColumnarToRow (106) + : +- CometProject (105) + : +- CometFilter (104) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (103) + +- CometSort (182) + +- CometColumnarExchange (181) + +- * HashAggregate (180) + +- * HashAggregate (179) + +- * Project (178) + +- * BroadcastHashJoin Inner BuildRight (177) + :- * Project (175) + : +- * BroadcastHashJoin Inner BuildRight (174) + : :- * Project (172) + : : +- * BroadcastHashJoin Inner BuildRight (171) + : : :- * Project (169) + : : : +- * BroadcastHashJoin Inner BuildRight (168) + : : : :- * Project (166) + : : : : +- * BroadcastHashJoin Inner BuildRight (165) + : : : : :- * Project (163) + : : : : : +- * BroadcastHashJoin Inner BuildRight (162) + : : : : : :- * Project (160) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (159) + : : : : : : :- * Project (157) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) + : : : : : : : :- * Project (154) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) + : : : : : : : : :- * Project (151) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) + : : : : : : : : : :- * Project (148) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) + : : : : : : : : : : :- * Project (145) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) + : : : : : : : : : : : :- * Project (142) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) + : : : : : : : : : : : : :- * Project (139) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) + : : : : : : : : : : : : : :- * Project (136) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) + : : : : : : : : : : : : : : :- * CometColumnarToRow (133) + : : : : : : : : : : : : : : : +- CometProject (132) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (131) + : : : : : : : : : : : : : : : :- CometSort (125) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (124) + : : : : : : : : : : : : : : : : +- * Project (123) + : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (122) + : : : : : : : : : : : : : : : : :- BroadcastExchange (117) + : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (116) + : : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (114) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (121) + : : : : : : : : : : : : : : : : +- CometProject (120) + : : : : : : : : : : : : : : : : +- CometFilter (119) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (118) + : : : : : : : : : : : : : : : +- CometSort (130) + : : : : : : : : : : : : : : : +- CometProject (129) + : : : : : : : : : : : : : : : +- CometFilter (128) + : : : : : : : : : : : : : : : +- CometHashAggregate (127) + : : : : : : : : : : : : : : : +- ReusedExchange (126) + : : : : : : : : : : : : : : +- ReusedExchange (134) + : : : : : : : : : : : : : +- ReusedExchange (137) + : : : : : : : : : : : : +- ReusedExchange (140) + : : : : : : : : : : : +- ReusedExchange (143) + : : : : : : : : : : +- ReusedExchange (146) + : : : : : : : : : +- ReusedExchange (149) + : : : : : : : : +- ReusedExchange (152) + : : : : : : : +- ReusedExchange (155) + : : : : : : +- ReusedExchange (158) + : : : : : +- ReusedExchange (161) + : : : : +- ReusedExchange (164) + : : : +- ReusedExchange (167) + : : +- ReusedExchange (170) + : +- ReusedExchange (173) + +- ReusedExchange (176) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -189,823 +200,884 @@ ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(6) CometProject +(7) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft +(8) CometColumnarToRow +Input [2]: [sr_item_sk#14, sr_ticket_number#15] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 2] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] +Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 2] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(11) CometColumnarExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(12) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(15) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(16) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(15) CometSort +(17) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(21) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(20) CometSort +(22) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(21) CometSortMergeJoin +(23) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(22) CometProject +(24) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(25) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(24) CometExchange +(26) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(25) CometHashAggregate +(27) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometFilter +(28) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(27) CometProject +(29) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(28) CometSort +(30) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(29) CometSortMergeJoin +(31) CometSortMergeJoin Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [1]: [cs_item_sk#17] Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(30) CometProject +(32) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(33) CometColumnarToRow [codegen id : 18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 191] +Output [2]: [d_date_sk#32, d_year#33] -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight +(35) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#32] +Join type: Inner +Join condition: None -(35) CometProject +(36) Project [codegen id : 18] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] ReadSchema: struct -(37) CometFilter +(38) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) -(38) CometProject +(39) CometProject Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] -(39) CometBroadcastExchange +(40) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight +(41) BroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(42) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#34] +Join type: Inner +Join condition: None -(41) CometProject +(43) Project [codegen id : 18] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(43) CometFilter +(45) CometFilter Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) -(44) CometBroadcastExchange +(46) CometColumnarToRow [codegen id : 5] Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(46) CometProject +(48) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#38] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter +(51) CometFilter Input [2]: [d_date_sk#44, d_year#45] Condition : isnotnull(d_date_sk#44) -(49) CometBroadcastExchange +(52) CometColumnarToRow [codegen id : 6] +Input [2]: [d_date_sk#44, d_year#45] + +(53) BroadcastExchange Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight +(54) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_first_sales_date_sk#43] +Right keys [1]: [d_date_sk#44] +Join type: Inner +Join condition: None -(51) CometProject +(55) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -(52) ReusedExchange [Reuses operator id: 49] +(56) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#46, d_year#47] -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight +(57) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_first_shipto_date_sk#42] +Right keys [1]: [d_date_sk#46] +Join type: Inner +Join condition: None -(54) CometProject +(58) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#48, cd_marital_status#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(56) CometFilter +(60) CometFilter Input [2]: [cd_demo_sk#48, cd_marital_status#49] Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) -(57) CometProject +(61) CometProject Input [2]: [cd_demo_sk#48, cd_marital_status#49] Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] -(58) CometBroadcastExchange +(62) CometColumnarToRow [codegen id : 8] Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight +(63) BroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(64) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#48] +Join type: Inner +Join condition: None -(60) CometProject +(65) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -(61) ReusedExchange [Reuses operator id: 58] +(66) ReusedExchange [Reuses operator id: 63] Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight +(67) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_cdemo_sk#39] +Right keys [1]: [cd_demo_sk#51] +Join type: Inner +Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) -(63) CometProject +(68) Project [codegen id : 18] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(65) CometFilter +(70) CometFilter Input [1]: [p_promo_sk#53] Condition : isnotnull(p_promo_sk#53) -(66) CometBroadcastExchange +(71) CometColumnarToRow [codegen id : 10] Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight +(72) BroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(68) CometProject +(73) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#53] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(70) CometFilter +(76) CometFilter Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) -(71) CometBroadcastExchange +(77) CometColumnarToRow [codegen id : 11] Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight +(78) BroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] + +(79) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#54] +Join type: Inner +Join condition: None -(73) CometProject +(80) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -(74) ReusedExchange [Reuses operator id: 71] +(81) ReusedExchange [Reuses operator id: 78] Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight +(82) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_hdemo_sk#40] +Right keys [1]: [hd_demo_sk#56] +Join type: Inner +Join condition: None -(76) CometProject +(83) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(78) CometFilter +(85) CometFilter Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Condition : isnotnull(ca_address_sk#58) -(79) CometProject +(86) CometProject Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] -(80) CometBroadcastExchange +(87) CometColumnarToRow [codegen id : 13] +Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] + +(88) BroadcastExchange Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight +(89) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#58] +Join type: Inner +Join condition: None -(82) CometProject +(90) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(83) ReusedExchange [Reuses operator id: 80] +(91) ReusedExchange [Reuses operator id: 88] Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight +(92) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#65] +Join type: Inner +Join condition: None -(85) CometProject +(93) Project [codegen id : 18] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(87) CometFilter +(95) CometFilter Input [1]: [ib_income_band_sk#70] Condition : isnotnull(ib_income_band_sk#70) -(88) CometBroadcastExchange +(96) CometColumnarToRow [codegen id : 15] Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight +(97) BroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(90) CometProject +(98) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [hd_income_band_sk#55] +Right keys [1]: [ib_income_band_sk#70] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 18] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(91) ReusedExchange [Reuses operator id: 88] +(100) ReusedExchange [Reuses operator id: 97] Output [1]: [ib_income_band_sk#71] -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight +(101) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [hd_income_band_sk#57] +Right keys [1]: [ib_income_band_sk#71] +Join type: Inner +Join condition: None -(93) CometProject +(102) Project [codegen id : 18] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(95) CometFilter +(104) CometFilter Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) -(96) CometProject +(105) CometProject Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] -(97) CometBroadcastExchange +(106) CometColumnarToRow [codegen id : 17] Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight +(107) BroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +(108) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#72] +Join type: Inner +Join condition: None -(99) CometProject +(109) Project [codegen id : 18] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -(100) CometHashAggregate +(110) HashAggregate [codegen id : 18] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] +(111) HashAggregate [codegen id : 18] +Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#72 AS item_sk#90, s_store_name#35 AS store_name#91, s_zip#37 AS store_zip#92, ca_street_number#63 AS b_street_number#93, ca_street_name#60 AS b_streen_name#94, ca_city#61 AS b_city#95, ca_zip#64 AS b_zip#96, ca_street_number#66 AS c_street_number#97, ca_street_name#67 AS c_street_name#98, ca_city#68 AS c_city#99, ca_zip#69 AS c_zip#100, d_year#33 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(112) CometColumnarExchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] +(113) CometSort +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +(114) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) +(115) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) + +(116) CometColumnarToRow [codegen id : 19] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +(117) BroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +(118) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner +(119) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] +(121) CometColumnarToRow +Input [2]: [sr_item_sk#119, sr_ticket_number#120] -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] +(122) BroadcastHashJoin [codegen id : 20] +Left keys [2]: [ss_item_sk#106, ss_ticket_number#113] +Right keys [2]: [sr_item_sk#119, sr_ticket_number#120] +Join type: Inner +Join condition: None -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight +(123) Project [codegen id : 20] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +(124) CometColumnarExchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] +(125) CometSort +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight +(126) ReusedExchange [Reuses operator id: 26] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +(127) CometHashAggregate +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] +(128) CometFilter +Input [3]: [cs_item_sk#122, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +(129) CometProject +Input [3]: [cs_item_sk#122, sale#30, refund#31] +Arguments: [cs_item_sk#122], [cs_item_sk#122] + +(130) CometSort +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122], [cs_item_sk#122 ASC NULLS FIRST] + +(131) CometSortMergeJoin +Left output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [1]: [cs_item_sk#122] +Arguments: [ss_item_sk#106], [cs_item_sk#122], Inner + +(132) CometProject +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(133) CometColumnarToRow [codegen id : 36] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(134) ReusedExchange [Reuses operator id: 195] +Output [2]: [d_date_sk#130, d_year#131] + +(135) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] +Join type: Inner +Join condition: None + +(136) Project [codegen id : 36] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] + +(137) ReusedExchange [Reuses operator id: 41] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#37] + +(138) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] +Join type: Inner +Join condition: None + +(139) Project [codegen id : 36] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#37] + +(140) ReusedExchange [Reuses operator id: 47] +Output [6]: [c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] + +(141) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#134] +Join type: Inner +Join condition: None + +(142) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] + +(143) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#140, d_year#141] + +(144) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_first_sales_date_sk#139] +Right keys [1]: [d_date_sk#140] +Join type: Inner +Join condition: None + +(145) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139, d_date_sk#140, d_year#141] + +(146) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#142, d_year#143] + +(147) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_first_shipto_date_sk#138] +Right keys [1]: [d_date_sk#142] +Join type: Inner +Join condition: None + +(148) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141, d_date_sk#142, d_year#143] + +(149) ReusedExchange [Reuses operator id: 63] +Output [2]: [cd_demo_sk#144, cd_marital_status#50] + +(150) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#144] +Join type: Inner +Join condition: None + +(151) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_demo_sk#144, cd_marital_status#50] + +(152) ReusedExchange [Reuses operator id: 63] +Output [2]: [cd_demo_sk#145, cd_marital_status#52] + +(153) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_cdemo_sk#135] +Right keys [1]: [cd_demo_sk#145] +Join type: Inner +Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) + +(154) Project [codegen id : 36] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50, cd_demo_sk#145, cd_marital_status#52] + +(155) ReusedExchange [Reuses operator id: 72] +Output [1]: [p_promo_sk#146] + +(156) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#146] +Join type: Inner +Join condition: None + +(157) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, p_promo_sk#146] + +(158) ReusedExchange [Reuses operator id: 78] +Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] + +(159) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#147] +Join type: Inner +Join condition: None + +(160) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_demo_sk#147, hd_income_band_sk#148] + +(161) ReusedExchange [Reuses operator id: 78] +Output [2]: [hd_demo_sk#149, hd_income_band_sk#150] + +(162) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_hdemo_sk#136] +Right keys [1]: [hd_demo_sk#149] +Join type: Inner +Join condition: None + +(163) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_demo_sk#149, hd_income_band_sk#150] + +(164) ReusedExchange [Reuses operator id: 88] +Output [5]: [ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] + +(165) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#151] +Join type: Inner +Join condition: None + +(166) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] + +(167) ReusedExchange [Reuses operator id: 88] +Output [5]: [ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] + +(168) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_addr_sk#137] +Right keys [1]: [ca_address_sk#154] +Join type: Inner +Join condition: None + +(169) Project [codegen id : 36] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] + +(170) ReusedExchange [Reuses operator id: 97] +Output [1]: [ib_income_band_sk#157] + +(171) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [hd_income_band_sk#148] +Right keys [1]: [ib_income_band_sk#157] +Join type: Inner +Join condition: None + +(172) Project [codegen id : 36] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#157] + +(173) ReusedExchange [Reuses operator id: 97] +Output [1]: [ib_income_band_sk#158] + +(174) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [hd_income_band_sk#150] +Right keys [1]: [ib_income_band_sk#158] +Join type: Inner +Join condition: None + +(175) Project [codegen id : 36] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#158] + +(176) ReusedExchange [Reuses operator id: 107] +Output [2]: [i_item_sk#159, i_product_name#76] + +(177) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#159] +Join type: Inner +Join condition: None + +(178) Project [codegen id : 36] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] + +(179) HashAggregate [codegen id : 36] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#160, sum#161, sum#162] +Results [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] + +(180) HashAggregate [codegen id : 36] +Input [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] +Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#159 AS item_sk#166, s_store_name#133 AS store_name#167, s_zip#37 AS store_zip#168, d_year#131 AS syear#169, count(1)#85 AS cnt#170, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#171, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#172, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#173] + +(181) CometColumnarExchange +Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: hashpartitioning(item_sk#166, store_name#167, store_zip#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] + +(182) CometSort +Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173], [item_sk#166 ASC NULLS FIRST, store_name#167 ASC NULLS FIRST, store_zip#168 ASC NULLS FIRST] + +(183) CometSortMergeJoin +Left output [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Right output [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#166, store_name#167, store_zip#168], Inner, (cnt#170 <= cnt#102) + +(184) CometProject +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] + +(185) CometExchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=19] + +(186) CometSort +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST] + +(187) CometColumnarToRow [codegen id : 37] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) +BroadcastExchange (191) ++- * CometColumnarToRow (190) + +- CometFilter (189) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (188) -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(188) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(178) CometFilter +(189) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(190) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(180) BroadcastExchange +(191) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) +Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (195) ++- * CometColumnarToRow (194) + +- CometFilter (193) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (192) -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] +(192) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) +(193) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] +(194) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(195) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index 059acab385..eb776d497d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -4,244 +4,280 @@ CometColumnarToRow +- 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 - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometSortMergeJoin + : : : : : : : : : : : : : : : : :- CometSort + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- 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 + : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : +- CometSortMergeJoin + : : : : : : : : : : : : : : : :- CometSort + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- 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 + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 138 out of 242 eligible operators (57%). Final plan contains 39 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt index 46fe063430..bc50f023fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (37) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] @@ -6,187 +6,260 @@ WholeStageCodegen (1) CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometExchange [ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (18) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (10) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometFilter [ib_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (17) + CometColumnarToRow + InputAdapter + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometExchange [ss_item_sk] #20 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + CometColumnarExchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (36) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #19 + WholeStageCodegen (20) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [p_promo_sk] #13 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 077daeca3a..753820f88a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,180 +1,191 @@ == Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) +* CometColumnarToRow (187) ++- CometSort (186) + +- CometExchange (185) + +- CometProject (184) + +- CometSortMergeJoin (183) + :- CometSort (113) + : +- CometColumnarExchange (112) + : +- * HashAggregate (111) + : +- * HashAggregate (110) + : +- * Project (109) + : +- * BroadcastHashJoin Inner BuildRight (108) + : :- * Project (102) + : : +- * BroadcastHashJoin Inner BuildRight (101) + : : :- * Project (99) + : : : +- * BroadcastHashJoin Inner BuildRight (98) + : : : :- * Project (93) + : : : : +- * BroadcastHashJoin Inner BuildRight (92) + : : : : :- * Project (90) + : : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : : :- * Project (83) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (82) + : : : : : : :- * Project (80) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : : : : :- * Project (74) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : : : : : : :- * Project (68) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (67) + : : : : : : : : : :- * Project (65) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) + : : : : : : : : : : :- * Project (58) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) + : : : : : : : : : : : :- * Project (55) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : : : : : : : : :- * Project (49) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : : : : : : : : : : :- * Project (43) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (33) + : : : : : : : : : : : : : : : : +- CometProject (32) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) + : : : : : : : : : : : : : : : : :- CometSort (12) + : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) + : : : : : : : : : : : : : : : : : +- * Project (10) + : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) + : : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometProject (7) + : : : : : : : : : : : : : : : : : +- CometFilter (6) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : : : : : : : : : : : : +- CometSort (30) + : : : : : : : : : : : : : : : : +- CometProject (29) + : : : : : : : : : : : : : : : : +- CometFilter (28) + : : : : : : : : : : : : : : : : +- CometHashAggregate (27) + : : : : : : : : : : : : : : : : +- CometExchange (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometProject (24) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) + : : : : : : : : : : : : : : : : :- CometSort (17) + : : : : : : : : : : : : : : : : : +- CometExchange (16) + : : : : : : : : : : : : : : : : : +- CometProject (15) + : : : : : : : : : : : : : : : : : +- CometFilter (14) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : : : : : : : : : : : : : : : +- CometSort (22) + : : : : : : : : : : : : : : : : +- CometExchange (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (41) + : : : : : : : : : : : : : : +- * CometColumnarToRow (40) + : : : : : : : : : : : : : : +- CometProject (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (47) + : : : : : : : : : : : : : +- * CometColumnarToRow (46) + : : : : : : : : : : : : : +- CometFilter (45) + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (44) + : : : : : : : : : : : : +- BroadcastExchange (53) + : : : : : : : : : : : : +- * CometColumnarToRow (52) + : : : : : : : : : : : : +- CometFilter (51) + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) + : : : : : : : : : : : +- ReusedExchange (56) + : : : : : : : : : : +- BroadcastExchange (63) + : : : : : : : : : : +- * CometColumnarToRow (62) + : : : : : : : : : : +- CometProject (61) + : : : : : : : : : : +- CometFilter (60) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (59) + : : : : : : : : : +- ReusedExchange (66) + : : : : : : : : +- BroadcastExchange (72) + : : : : : : : : +- * CometColumnarToRow (71) + : : : : : : : : +- CometFilter (70) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (69) + : : : : : : : +- BroadcastExchange (78) + : : : : : : : +- * CometColumnarToRow (77) + : : : : : : : +- CometFilter (76) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (75) + : : : : : : +- ReusedExchange (81) + : : : : : +- BroadcastExchange (88) + : : : : : +- * CometColumnarToRow (87) + : : : : : +- CometProject (86) + : : : : : +- CometFilter (85) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (84) + : : : : +- ReusedExchange (91) + : : : +- BroadcastExchange (97) + : : : +- * CometColumnarToRow (96) + : : : +- CometFilter (95) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (94) + : : +- ReusedExchange (100) + : +- BroadcastExchange (107) + : +- * CometColumnarToRow (106) + : +- CometProject (105) + : +- CometFilter (104) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (103) + +- CometSort (182) + +- CometColumnarExchange (181) + +- * HashAggregate (180) + +- * HashAggregate (179) + +- * Project (178) + +- * BroadcastHashJoin Inner BuildRight (177) + :- * Project (175) + : +- * BroadcastHashJoin Inner BuildRight (174) + : :- * Project (172) + : : +- * BroadcastHashJoin Inner BuildRight (171) + : : :- * Project (169) + : : : +- * BroadcastHashJoin Inner BuildRight (168) + : : : :- * Project (166) + : : : : +- * BroadcastHashJoin Inner BuildRight (165) + : : : : :- * Project (163) + : : : : : +- * BroadcastHashJoin Inner BuildRight (162) + : : : : : :- * Project (160) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (159) + : : : : : : :- * Project (157) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) + : : : : : : : :- * Project (154) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) + : : : : : : : : :- * Project (151) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) + : : : : : : : : : :- * Project (148) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) + : : : : : : : : : : :- * Project (145) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) + : : : : : : : : : : : :- * Project (142) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) + : : : : : : : : : : : : :- * Project (139) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) + : : : : : : : : : : : : : :- * Project (136) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) + : : : : : : : : : : : : : : :- * CometColumnarToRow (133) + : : : : : : : : : : : : : : : +- CometProject (132) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (131) + : : : : : : : : : : : : : : : :- CometSort (125) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (124) + : : : : : : : : : : : : : : : : +- * Project (123) + : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (122) + : : : : : : : : : : : : : : : : :- BroadcastExchange (117) + : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (116) + : : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (114) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (121) + : : : : : : : : : : : : : : : : +- CometProject (120) + : : : : : : : : : : : : : : : : +- CometFilter (119) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (118) + : : : : : : : : : : : : : : : +- CometSort (130) + : : : : : : : : : : : : : : : +- CometProject (129) + : : : : : : : : : : : : : : : +- CometFilter (128) + : : : : : : : : : : : : : : : +- CometHashAggregate (127) + : : : : : : : : : : : : : : : +- ReusedExchange (126) + : : : : : : : : : : : : : : +- ReusedExchange (134) + : : : : : : : : : : : : : +- ReusedExchange (137) + : : : : : : : : : : : : +- ReusedExchange (140) + : : : : : : : : : : : +- ReusedExchange (143) + : : : : : : : : : : +- ReusedExchange (146) + : : : : : : : : : +- ReusedExchange (149) + : : : : : : : : +- ReusedExchange (152) + : : : : : : : +- ReusedExchange (155) + : : : : : : +- ReusedExchange (158) + : : : : : +- ReusedExchange (161) + : : : : +- ReusedExchange (164) + : : : +- ReusedExchange (167) + : : +- ReusedExchange (170) + : +- ReusedExchange (173) + +- ReusedExchange (176) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -189,823 +200,884 @@ ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(6) CometProject +(7) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft +(8) CometColumnarToRow +Input [2]: [sr_item_sk#14, sr_ticket_number#15] -(8) CometProject +(9) BroadcastHashJoin [codegen id : 2] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] +Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 2] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(9) CometExchange +(11) CometColumnarExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) CometSort +(12) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(13) CometProject +(15) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(14) CometExchange +(16) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(15) CometSort +(17) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(17) CometFilter +(19) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(18) CometProject +(20) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(19) CometExchange +(21) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(20) CometSort +(22) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(21) CometSortMergeJoin +(23) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(22) CometProject +(24) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(23) CometHashAggregate +(25) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(24) CometExchange +(26) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(25) CometHashAggregate +(27) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometFilter +(28) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(27) CometProject +(29) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(28) CometSort +(30) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(29) CometSortMergeJoin +(31) CometSortMergeJoin Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [1]: [cs_item_sk#17] Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(30) CometProject +(32) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) +(33) CometColumnarToRow [codegen id : 18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] +(34) ReusedExchange [Reuses operator id: 191] +Output [2]: [d_date_sk#32, d_year#33] -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight +(35) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#32] +Join type: Inner +Join condition: None -(35) CometProject +(36) Project [codegen id : 18] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] ReadSchema: struct -(37) CometFilter +(38) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) -(38) CometProject +(39) CometProject Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] -(39) CometBroadcastExchange +(40) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight +(41) BroadcastExchange +Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(42) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#34] +Join type: Inner +Join condition: None -(41) CometProject +(43) Project [codegen id : 18] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(43) CometFilter +(45) CometFilter Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) -(44) CometBroadcastExchange +(46) CometColumnarToRow [codegen id : 5] Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(46) CometProject +(48) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#38] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter +(51) CometFilter Input [2]: [d_date_sk#44, d_year#45] Condition : isnotnull(d_date_sk#44) -(49) CometBroadcastExchange +(52) CometColumnarToRow [codegen id : 6] +Input [2]: [d_date_sk#44, d_year#45] + +(53) BroadcastExchange Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight +(54) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_first_sales_date_sk#43] +Right keys [1]: [d_date_sk#44] +Join type: Inner +Join condition: None -(51) CometProject +(55) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -(52) ReusedExchange [Reuses operator id: 49] +(56) ReusedExchange [Reuses operator id: 53] Output [2]: [d_date_sk#46, d_year#47] -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight +(57) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_first_shipto_date_sk#42] +Right keys [1]: [d_date_sk#46] +Join type: Inner +Join condition: None -(54) CometProject +(58) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#48, cd_marital_status#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(56) CometFilter +(60) CometFilter Input [2]: [cd_demo_sk#48, cd_marital_status#49] Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) -(57) CometProject +(61) CometProject Input [2]: [cd_demo_sk#48, cd_marital_status#49] Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] -(58) CometBroadcastExchange +(62) CometColumnarToRow [codegen id : 8] Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight +(63) BroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(64) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#48] +Join type: Inner +Join condition: None -(60) CometProject +(65) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -(61) ReusedExchange [Reuses operator id: 58] +(66) ReusedExchange [Reuses operator id: 63] Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight +(67) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_cdemo_sk#39] +Right keys [1]: [cd_demo_sk#51] +Join type: Inner +Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) -(63) CometProject +(68) Project [codegen id : 18] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(65) CometFilter +(70) CometFilter Input [1]: [p_promo_sk#53] Condition : isnotnull(p_promo_sk#53) -(66) CometBroadcastExchange +(71) CometColumnarToRow [codegen id : 10] Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight +(72) BroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(68) CometProject +(73) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#53] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(70) CometFilter +(76) CometFilter Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) -(71) CometBroadcastExchange +(77) CometColumnarToRow [codegen id : 11] Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight +(78) BroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] + +(79) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#54] +Join type: Inner +Join condition: None -(73) CometProject +(80) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -(74) ReusedExchange [Reuses operator id: 71] +(81) ReusedExchange [Reuses operator id: 78] Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight +(82) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_hdemo_sk#40] +Right keys [1]: [hd_demo_sk#56] +Join type: Inner +Join condition: None -(76) CometProject +(83) Project [codegen id : 18] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(78) CometFilter +(85) CometFilter Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Condition : isnotnull(ca_address_sk#58) -(79) CometProject +(86) CometProject Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] -(80) CometBroadcastExchange +(87) CometColumnarToRow [codegen id : 13] +Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] + +(88) BroadcastExchange Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight +(89) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#58] +Join type: Inner +Join condition: None -(82) CometProject +(90) Project [codegen id : 18] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(83) ReusedExchange [Reuses operator id: 80] +(91) ReusedExchange [Reuses operator id: 88] Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight +(92) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#65] +Join type: Inner +Join condition: None -(85) CometProject +(93) Project [codegen id : 18] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(87) CometFilter +(95) CometFilter Input [1]: [ib_income_band_sk#70] Condition : isnotnull(ib_income_band_sk#70) -(88) CometBroadcastExchange +(96) CometColumnarToRow [codegen id : 15] Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight +(97) BroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(90) CometProject +(98) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [hd_income_band_sk#55] +Right keys [1]: [ib_income_band_sk#70] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 18] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(91) ReusedExchange [Reuses operator id: 88] +(100) ReusedExchange [Reuses operator id: 97] Output [1]: [ib_income_band_sk#71] -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight +(101) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [hd_income_band_sk#57] +Right keys [1]: [ib_income_band_sk#71] +Join type: Inner +Join condition: None -(93) CometProject +(102) Project [codegen id : 18] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(95) CometFilter +(104) CometFilter Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) -(96) CometProject +(105) CometProject Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] -(97) CometBroadcastExchange +(106) CometColumnarToRow [codegen id : 17] Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight +(107) BroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +(108) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#72] +Join type: Inner +Join condition: None -(99) CometProject +(109) Project [codegen id : 18] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -(100) CometHashAggregate +(110) HashAggregate [codegen id : 18] Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] +(111) HashAggregate [codegen id : 18] +Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#72 AS item_sk#90, s_store_name#35 AS store_name#91, s_zip#37 AS store_zip#92, ca_street_number#63 AS b_street_number#93, ca_street_name#60 AS b_streen_name#94, ca_city#61 AS b_city#95, ca_zip#64 AS b_zip#96, ca_street_number#66 AS c_street_number#97, ca_street_name#67 AS c_street_name#98, ca_city#68 AS c_city#99, ca_zip#69 AS c_zip#100, d_year#33 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(112) CometColumnarExchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] +(113) CometSort +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +(114) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) +(115) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) + +(116) CometColumnarToRow [codegen id : 19] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +(117) BroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +(118) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner +(119) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) (120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] +(121) CometColumnarToRow +Input [2]: [sr_item_sk#119, sr_ticket_number#120] -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] +(122) BroadcastHashJoin [codegen id : 20] +Left keys [2]: [ss_item_sk#106, ss_ticket_number#113] +Right keys [2]: [sr_item_sk#119, sr_ticket_number#120] +Join type: Inner +Join condition: None -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight +(123) Project [codegen id : 20] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +(124) CometColumnarExchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] +(125) CometSort +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight +(126) ReusedExchange [Reuses operator id: 26] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +(127) CometHashAggregate +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] +(128) CometFilter +Input [3]: [cs_item_sk#122, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +(129) CometProject +Input [3]: [cs_item_sk#122, sale#30, refund#31] +Arguments: [cs_item_sk#122], [cs_item_sk#122] + +(130) CometSort +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122], [cs_item_sk#122 ASC NULLS FIRST] + +(131) CometSortMergeJoin +Left output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [1]: [cs_item_sk#122] +Arguments: [ss_item_sk#106], [cs_item_sk#122], Inner + +(132) CometProject +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(133) CometColumnarToRow [codegen id : 36] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(134) ReusedExchange [Reuses operator id: 195] +Output [2]: [d_date_sk#130, d_year#131] + +(135) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#130] +Join type: Inner +Join condition: None + +(136) Project [codegen id : 36] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] + +(137) ReusedExchange [Reuses operator id: 41] +Output [3]: [s_store_sk#132, s_store_name#133, s_zip#37] + +(138) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#132] +Join type: Inner +Join condition: None + +(139) Project [codegen id : 36] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#37] + +(140) ReusedExchange [Reuses operator id: 47] +Output [6]: [c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] + +(141) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#134] +Join type: Inner +Join condition: None + +(142) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] + +(143) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#140, d_year#141] + +(144) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_first_sales_date_sk#139] +Right keys [1]: [d_date_sk#140] +Join type: Inner +Join condition: None + +(145) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139, d_date_sk#140, d_year#141] + +(146) ReusedExchange [Reuses operator id: 53] +Output [2]: [d_date_sk#142, d_year#143] + +(147) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_first_shipto_date_sk#138] +Right keys [1]: [d_date_sk#142] +Join type: Inner +Join condition: None + +(148) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141, d_date_sk#142, d_year#143] + +(149) ReusedExchange [Reuses operator id: 63] +Output [2]: [cd_demo_sk#144, cd_marital_status#50] + +(150) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#144] +Join type: Inner +Join condition: None + +(151) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_demo_sk#144, cd_marital_status#50] + +(152) ReusedExchange [Reuses operator id: 63] +Output [2]: [cd_demo_sk#145, cd_marital_status#52] + +(153) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_cdemo_sk#135] +Right keys [1]: [cd_demo_sk#145] +Join type: Inner +Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) + +(154) Project [codegen id : 36] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50, cd_demo_sk#145, cd_marital_status#52] + +(155) ReusedExchange [Reuses operator id: 72] +Output [1]: [p_promo_sk#146] + +(156) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#146] +Join type: Inner +Join condition: None + +(157) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, p_promo_sk#146] + +(158) ReusedExchange [Reuses operator id: 78] +Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] + +(159) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#147] +Join type: Inner +Join condition: None + +(160) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_demo_sk#147, hd_income_band_sk#148] + +(161) ReusedExchange [Reuses operator id: 78] +Output [2]: [hd_demo_sk#149, hd_income_band_sk#150] + +(162) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_hdemo_sk#136] +Right keys [1]: [hd_demo_sk#149] +Join type: Inner +Join condition: None + +(163) Project [codegen id : 36] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_demo_sk#149, hd_income_band_sk#150] + +(164) ReusedExchange [Reuses operator id: 88] +Output [5]: [ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] + +(165) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#151] +Join type: Inner +Join condition: None + +(166) Project [codegen id : 36] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] + +(167) ReusedExchange [Reuses operator id: 88] +Output [5]: [ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] + +(168) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [c_current_addr_sk#137] +Right keys [1]: [ca_address_sk#154] +Join type: Inner +Join condition: None + +(169) Project [codegen id : 36] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] + +(170) ReusedExchange [Reuses operator id: 97] +Output [1]: [ib_income_band_sk#157] + +(171) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [hd_income_band_sk#148] +Right keys [1]: [ib_income_band_sk#157] +Join type: Inner +Join condition: None + +(172) Project [codegen id : 36] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#157] + +(173) ReusedExchange [Reuses operator id: 97] +Output [1]: [ib_income_band_sk#158] + +(174) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [hd_income_band_sk#150] +Right keys [1]: [ib_income_band_sk#158] +Join type: Inner +Join condition: None + +(175) Project [codegen id : 36] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#158] + +(176) ReusedExchange [Reuses operator id: 107] +Output [2]: [i_item_sk#159, i_product_name#76] + +(177) BroadcastHashJoin [codegen id : 36] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#159] +Join type: Inner +Join condition: None + +(178) Project [codegen id : 36] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] + +(179) HashAggregate [codegen id : 36] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#160, sum#161, sum#162] +Results [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] + +(180) HashAggregate [codegen id : 36] +Input [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] +Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#159 AS item_sk#166, s_store_name#133 AS store_name#167, s_zip#37 AS store_zip#168, d_year#131 AS syear#169, count(1)#85 AS cnt#170, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#171, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#172, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#173] + +(181) CometColumnarExchange +Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: hashpartitioning(item_sk#166, store_name#167, store_zip#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] + +(182) CometSort +Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173], [item_sk#166 ASC NULLS FIRST, store_name#167 ASC NULLS FIRST, store_zip#168 ASC NULLS FIRST] + +(183) CometSortMergeJoin +Left output [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Right output [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#166, store_name#167, store_zip#168], Inner, (cnt#170 <= cnt#102) + +(184) CometProject +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] + +(185) CometExchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=19] + +(186) CometSort +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST] + +(187) CometColumnarToRow [codegen id : 37] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) +BroadcastExchange (191) ++- * CometColumnarToRow (190) + +- CometFilter (189) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (188) -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(188) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(178) CometFilter +(189) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(190) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(180) BroadcastExchange +(191) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) +Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (195) ++- * CometColumnarToRow (194) + +- CometFilter (193) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (192) -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] +(192) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#130, d_year#131] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) +(193) CometFilter +Input [2]: [d_date_sk#130, d_year#131] +Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] +(194) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#130, d_year#131] -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(195) BroadcastExchange +Input [2]: [d_date_sk#130, d_year#131] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt index 059acab385..eb776d497d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt @@ -4,244 +4,280 @@ CometColumnarToRow +- 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 - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometSortMergeJoin + : : : : : : : : : : : : : : : : :- CometSort + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- 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 + : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Project + : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : :- Project + : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : :- Project + : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : :- Project + : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : +- CometSortMergeJoin + : : : : : : : : : : : : : : : :- CometSort + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- 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 + : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometProject + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 138 out of 242 eligible operators (57%). Final plan contains 39 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 46fe063430..bc50f023fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (37) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] @@ -6,187 +6,260 @@ WholeStageCodegen (1) CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometExchange [ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + CometColumnarExchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (18) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (10) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (13) + CometColumnarToRow + InputAdapter + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometFilter [ib_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (17) + CometColumnarToRow + InputAdapter + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometExchange [ss_item_sk] #20 - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + CometColumnarExchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (36) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #19 + WholeStageCodegen (20) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [p_promo_sk] #13 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt index 46f670a0bd..ffe7401317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt @@ -1,78 +1,96 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) +* CometColumnarToRow (92) ++- CometTakeOrderedAndProject (91) + +- RowToColumnar (90) + +- * Filter (89) + +- Window (88) + +- WindowGroupLimit (87) + +- * CometColumnarToRow (86) + +- CometSort (85) + +- CometColumnarExchange (84) + +- WindowGroupLimit (83) + +- * Sort (82) + +- Union (81) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (14) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- ReusedExchange (25) + :- * HashAggregate (38) + : +- * CometColumnarToRow (37) + : +- CometColumnarExchange (36) + : +- * HashAggregate (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + :- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- ReusedExchange (39) + :- * HashAggregate (52) + : +- * CometColumnarToRow (51) + : +- CometColumnarExchange (50) + : +- * HashAggregate (49) + : +- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- ReusedExchange (46) + :- * HashAggregate (59) + : +- * CometColumnarToRow (58) + : +- CometColumnarExchange (57) + : +- * HashAggregate (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- ReusedExchange (53) + :- * HashAggregate (66) + : +- * CometColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- * HashAggregate (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (61) + : +- ReusedExchange (60) + :- * HashAggregate (73) + : +- * CometColumnarToRow (72) + : +- CometColumnarExchange (71) + : +- * HashAggregate (70) + : +- * HashAggregate (69) + : +- * CometColumnarToRow (68) + : +- ReusedExchange (67) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -87,359 +105,452 @@ ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(4) ReusedExchange [Reuses operator id: 97] +Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +(6) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_store_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) +(8) CometFilter +Input [2]: [s_store_sk#11, s_store_id#12] +Condition : isnotnull(s_store_sk#11) -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] +(9) CometProject +Input [2]: [s_store_sk#11, s_store_id#12] +Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] +(10) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#11, s_store_id#13] -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight +(11) BroadcastExchange +Input [2]: [s_store_sk#11, s_store_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +(13) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +(16) CometProject +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight +(17) CometColumnarToRow [codegen id : 3] +Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +(18) BroadcastExchange +Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#14] +Join type: Inner +Join condition: None -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] +(20) Project [codegen id : 4] +Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#43)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#43)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] +(21) HashAggregate [codegen id : 4] +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [partial_sum(sumsales#53)] +(22) CometColumnarExchange +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(23) CometColumnarToRow [codegen id : 5] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [sum(sumsales#53)] +(24) HashAggregate [codegen id : 5] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] +Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] + +(25) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] + +(26) CometColumnarToRow [codegen id : 10] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] + +(27) HashAggregate [codegen id : 10] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] +Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] + +(28) HashAggregate [codegen id : 10] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] +Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] + +(29) CometColumnarExchange +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 11] +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] + +(31) HashAggregate [codegen id : 11] +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] +Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] + +(32) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] + +(33) CometColumnarToRow [codegen id : 16] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] + +(34) HashAggregate [codegen id : 16] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] +Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] + +(35) HashAggregate [codegen id : 16] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] +Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] +Functions [1]: [partial_sum(sumsales#59)] +Aggregate Attributes [2]: [sum#60, isEmpty#61] +Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] + +(36) CometColumnarExchange +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(37) CometColumnarToRow [codegen id : 17] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] + +(38) HashAggregate [codegen id : 17] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] +Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] +Functions [1]: [sum(sumsales#59)] +Aggregate Attributes [1]: [sum(sumsales#59)#64] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#65, null AS s_store_id#66, sum(sumsales#59)#64 AS sumsales#67] (39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#63)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#63)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#73)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#73)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#83] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#83)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#83)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#93] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#93)] - -(57) CometExchange -Input [3]: [i_category#22, sum#94, isEmpty#95] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#94, isEmpty#95] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#93)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#103] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] + +(40) CometColumnarToRow [codegen id : 22] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] + +(41) HashAggregate [codegen id : 22] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] +Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] + +(42) HashAggregate [codegen id : 22] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] +Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] +Functions [1]: [partial_sum(sumsales#75)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] + +(43) CometColumnarExchange +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(44) CometColumnarToRow [codegen id : 23] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] + +(45) HashAggregate [codegen id : 23] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] +Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] +Functions [1]: [sum(sumsales#75)] +Aggregate Attributes [1]: [sum(sumsales#75)#80] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, null AS d_qoy#81, null AS d_moy#82, null AS s_store_id#83, sum(sumsales#75)#80 AS sumsales#84] + +(46) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] + +(47) CometColumnarToRow [codegen id : 28] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] + +(48) HashAggregate [codegen id : 28] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] +Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] + +(49) HashAggregate [codegen id : 28] +Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] +Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] +Functions [1]: [partial_sum(sumsales#92)] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] + +(50) CometColumnarExchange +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(51) CometColumnarToRow [codegen id : 29] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] + +(52) HashAggregate [codegen id : 29] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] +Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] +Functions [1]: [sum(sumsales#92)] +Aggregate Attributes [1]: [sum(sumsales#92)#97] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#92)#97 AS sumsales#102] + +(53) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] + +(54) CometColumnarToRow [codegen id : 34] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] + +(55) HashAggregate [codegen id : 34] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] +Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] + +(56) HashAggregate [codegen id : 34] +Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] +Keys [3]: [i_category#21, i_class#20, i_brand#19] +Functions [1]: [partial_sum(sumsales#110)] +Aggregate Attributes [2]: [sum#111, isEmpty#112] +Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] + +(57) CometColumnarExchange +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(58) CometColumnarToRow [codegen id : 35] +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] + +(59) HashAggregate [codegen id : 35] +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] +Keys [3]: [i_category#21, i_class#20, i_brand#19] +Functions [1]: [sum(sumsales#110)] +Aggregate Attributes [1]: [sum(sumsales#110)#115] +Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, null AS d_year#117, null AS d_qoy#118, null AS d_moy#119, null AS s_store_id#120, sum(sumsales#110)#115 AS sumsales#121] + +(60) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] + +(61) CometColumnarToRow [codegen id : 40] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] + +(62) HashAggregate [codegen id : 40] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] +Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] + +(63) HashAggregate [codegen id : 40] +Input [3]: [i_category#21, i_class#20, sumsales#129] +Keys [2]: [i_category#21, i_class#20] +Functions [1]: [partial_sum(sumsales#129)] +Aggregate Attributes [2]: [sum#130, isEmpty#131] +Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] + +(64) CometColumnarExchange +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] +Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(65) CometColumnarToRow [codegen id : 41] +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] + +(66) HashAggregate [codegen id : 41] +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] +Keys [2]: [i_category#21, i_class#20] +Functions [1]: [sum(sumsales#129)] +Aggregate Attributes [1]: [sum(sumsales#129)#134] +Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#129)#134 AS sumsales#141] + +(67) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] + +(68) CometColumnarToRow [codegen id : 46] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] + +(69) HashAggregate [codegen id : 46] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] +Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] + +(70) HashAggregate [codegen id : 46] +Input [2]: [i_category#21, sumsales#149] +Keys [1]: [i_category#21] +Functions [1]: [partial_sum(sumsales#149)] +Aggregate Attributes [2]: [sum#150, isEmpty#151] +Results [3]: [i_category#21, sum#152, isEmpty#153] + +(71) CometColumnarExchange +Input [3]: [i_category#21, sum#152, isEmpty#153] +Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(72) CometColumnarToRow [codegen id : 47] +Input [3]: [i_category#21, sum#152, isEmpty#153] + +(73) HashAggregate [codegen id : 47] +Input [3]: [i_category#21, sum#152, isEmpty#153] +Keys [1]: [i_category#21] +Functions [1]: [sum(sumsales#149)] +Aggregate Attributes [1]: [sum(sumsales#149)#154] +Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i_product_name#157, null AS d_year#158, null AS d_qoy#159, null AS d_moy#160, null AS s_store_id#161, sum(sumsales#149)#154 AS sumsales#162] + +(74) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] + +(75) CometColumnarToRow [codegen id : 52] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] + +(76) HashAggregate [codegen id : 52] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] +Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] + +(77) HashAggregate [codegen id : 52] +Input [1]: [sumsales#170] Keys: [] -Functions [1]: [partial_sum(sumsales#103)] +Functions [1]: [partial_sum(sumsales#170)] +Aggregate Attributes [2]: [sum#171, isEmpty#172] +Results [2]: [sum#173, isEmpty#174] + +(78) CometColumnarExchange +Input [2]: [sum#173, isEmpty#174] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(62) CometExchange -Input [2]: [sum#104, isEmpty#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(79) CometColumnarToRow [codegen id : 53] +Input [2]: [sum#173, isEmpty#174] -(63) CometHashAggregate -Input [2]: [sum#104, isEmpty#105] +(80) HashAggregate [codegen id : 53] +Input [2]: [sum#173, isEmpty#174] Keys: [] -Functions [1]: [sum(sumsales#103)] +Functions [1]: [sum(sumsales#170)] +Aggregate Attributes [1]: [sum(sumsales#170)#175] +Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#170)#175 AS sumsales#184] + +(81) Union -(64) CometUnion -Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] -Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] -Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] +(82) Sort [codegen id : 54] +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 -(65) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] +(83) WindowGroupLimit +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +(84) CometColumnarExchange +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(67) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial +(85) CometSort +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] -(68) CometColumnarExchange -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(86) CometColumnarToRow [codegen id : 55] +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -(69) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] +(87) WindowGroupLimit +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +(88) Window +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] -(71) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final +(89) Filter [codegen id : 56] +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +Condition : (rk#185 <= 100) -(72) Window -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] +(90) RowToColumnar +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -(73) Filter [codegen id : 3] -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Condition : (rk#159 <= 100) +(91) CometTakeOrderedAndProject +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#28 ASC NULLS FIRST,i_class#29 ASC NULLS FIRST,i_brand#30 ASC NULLS FIRST,i_product_name#31 ASC NULLS FIRST,d_year#32 ASC NULLS FIRST,d_qoy#33 ASC NULLS FIRST,d_moy#34 ASC NULLS FIRST,s_store_id#35 ASC NULLS FIRST,sumsales#36 ASC NULLS FIRST,rk#185 ASC NULLS FIRST], output=[i_category#28,i_class#29,i_brand#30,i_product_name#31,d_year#32,d_qoy#33,d_moy#34,s_store_id#35,sumsales#36,rk#185]), [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185], 100, 0, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#185 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -(74) TakeOrderedAndProject -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +(92) CometColumnarToRow [codegen id : 57] +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (97) ++- * CometColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) +(94) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_month_seq#186 <= 1223)) AND isnotnull(d_date_sk#7)) -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(95) CometProject +Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(96) CometColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(97) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index da0a1c697a..275e74b65a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -1,298 +1,352 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- Sort + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 146 out of 285 eligible operators (51%). Final plan contains 65 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt index 6096ee1c25..a56fc90d3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt @@ -1,88 +1,151 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +WholeStageCodegen (57) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + RowToColumnar + WholeStageCodegen (56) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (55) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (54) + Sort [i_category,sumsales] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (22) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (29) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (28) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (35) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (34) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (41) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #11 + WholeStageCodegen (40) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #12 + WholeStageCodegen (46) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #13 + WholeStageCodegen (52) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt index 46f670a0bd..ffe7401317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt @@ -1,78 +1,96 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) +* CometColumnarToRow (92) ++- CometTakeOrderedAndProject (91) + +- RowToColumnar (90) + +- * Filter (89) + +- Window (88) + +- WindowGroupLimit (87) + +- * CometColumnarToRow (86) + +- CometSort (85) + +- CometColumnarExchange (84) + +- WindowGroupLimit (83) + +- * Sort (82) + +- Union (81) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * CometColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * CometColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * CometColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (14) + :- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- ReusedExchange (25) + :- * HashAggregate (38) + : +- * CometColumnarToRow (37) + : +- CometColumnarExchange (36) + : +- * HashAggregate (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- ReusedExchange (32) + :- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- ReusedExchange (39) + :- * HashAggregate (52) + : +- * CometColumnarToRow (51) + : +- CometColumnarExchange (50) + : +- * HashAggregate (49) + : +- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- ReusedExchange (46) + :- * HashAggregate (59) + : +- * CometColumnarToRow (58) + : +- CometColumnarExchange (57) + : +- * HashAggregate (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- ReusedExchange (53) + :- * HashAggregate (66) + : +- * CometColumnarToRow (65) + : +- CometColumnarExchange (64) + : +- * HashAggregate (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (61) + : +- ReusedExchange (60) + :- * HashAggregate (73) + : +- * CometColumnarToRow (72) + : +- CometColumnarExchange (71) + : +- * HashAggregate (70) + : +- * HashAggregate (69) + : +- * CometColumnarToRow (68) + : +- ReusedExchange (67) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -87,359 +105,452 @@ ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(3) CometColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(4) ReusedExchange [Reuses operator id: 97] +Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +(6) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_store_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) +(8) CometFilter +Input [2]: [s_store_sk#11, s_store_id#12] +Condition : isnotnull(s_store_sk#11) -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] +(9) CometProject +Input [2]: [s_store_sk#11, s_store_id#12] +Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] +(10) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#11, s_store_id#13] -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight +(11) BroadcastExchange +Input [2]: [s_store_sk#11, s_store_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +(13) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] + +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +(16) CometProject +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight +(17) CometColumnarToRow [codegen id : 3] +Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +(18) BroadcastExchange +Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] +(19) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#14] +Join type: Inner +Join condition: None -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] +(20) Project [codegen id : 4] +Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#43)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#43)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] +(21) HashAggregate [codegen id : 4] +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [partial_sum(sumsales#53)] +(22) CometColumnarExchange +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(23) CometColumnarToRow [codegen id : 5] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [sum(sumsales#53)] +(24) HashAggregate [codegen id : 5] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] +Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] + +(25) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] + +(26) CometColumnarToRow [codegen id : 10] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] + +(27) HashAggregate [codegen id : 10] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] +Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] + +(28) HashAggregate [codegen id : 10] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] +Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] + +(29) CometColumnarExchange +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(30) CometColumnarToRow [codegen id : 11] +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] + +(31) HashAggregate [codegen id : 11] +Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] +Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] + +(32) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] + +(33) CometColumnarToRow [codegen id : 16] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] + +(34) HashAggregate [codegen id : 16] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] +Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] + +(35) HashAggregate [codegen id : 16] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] +Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] +Functions [1]: [partial_sum(sumsales#59)] +Aggregate Attributes [2]: [sum#60, isEmpty#61] +Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] + +(36) CometColumnarExchange +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(37) CometColumnarToRow [codegen id : 17] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] + +(38) HashAggregate [codegen id : 17] +Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] +Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] +Functions [1]: [sum(sumsales#59)] +Aggregate Attributes [1]: [sum(sumsales#59)#64] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#65, null AS s_store_id#66, sum(sumsales#59)#64 AS sumsales#67] (39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#63)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#63)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#73)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#73)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#83] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#83)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#83)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#93] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#93)] - -(57) CometExchange -Input [3]: [i_category#22, sum#94, isEmpty#95] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#94, isEmpty#95] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#93)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#103] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] + +(40) CometColumnarToRow [codegen id : 22] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] + +(41) HashAggregate [codegen id : 22] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] +Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] + +(42) HashAggregate [codegen id : 22] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] +Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] +Functions [1]: [partial_sum(sumsales#75)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] + +(43) CometColumnarExchange +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(44) CometColumnarToRow [codegen id : 23] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] + +(45) HashAggregate [codegen id : 23] +Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] +Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] +Functions [1]: [sum(sumsales#75)] +Aggregate Attributes [1]: [sum(sumsales#75)#80] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, null AS d_qoy#81, null AS d_moy#82, null AS s_store_id#83, sum(sumsales#75)#80 AS sumsales#84] + +(46) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] + +(47) CometColumnarToRow [codegen id : 28] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] + +(48) HashAggregate [codegen id : 28] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] +Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] + +(49) HashAggregate [codegen id : 28] +Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] +Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] +Functions [1]: [partial_sum(sumsales#92)] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] + +(50) CometColumnarExchange +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(51) CometColumnarToRow [codegen id : 29] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] + +(52) HashAggregate [codegen id : 29] +Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] +Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] +Functions [1]: [sum(sumsales#92)] +Aggregate Attributes [1]: [sum(sumsales#92)#97] +Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#92)#97 AS sumsales#102] + +(53) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] + +(54) CometColumnarToRow [codegen id : 34] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] + +(55) HashAggregate [codegen id : 34] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] +Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] + +(56) HashAggregate [codegen id : 34] +Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] +Keys [3]: [i_category#21, i_class#20, i_brand#19] +Functions [1]: [partial_sum(sumsales#110)] +Aggregate Attributes [2]: [sum#111, isEmpty#112] +Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] + +(57) CometColumnarExchange +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] +Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(58) CometColumnarToRow [codegen id : 35] +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] + +(59) HashAggregate [codegen id : 35] +Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] +Keys [3]: [i_category#21, i_class#20, i_brand#19] +Functions [1]: [sum(sumsales#110)] +Aggregate Attributes [1]: [sum(sumsales#110)#115] +Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, null AS d_year#117, null AS d_qoy#118, null AS d_moy#119, null AS s_store_id#120, sum(sumsales#110)#115 AS sumsales#121] + +(60) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] + +(61) CometColumnarToRow [codegen id : 40] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] + +(62) HashAggregate [codegen id : 40] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] +Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] + +(63) HashAggregate [codegen id : 40] +Input [3]: [i_category#21, i_class#20, sumsales#129] +Keys [2]: [i_category#21, i_class#20] +Functions [1]: [partial_sum(sumsales#129)] +Aggregate Attributes [2]: [sum#130, isEmpty#131] +Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] + +(64) CometColumnarExchange +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] +Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(65) CometColumnarToRow [codegen id : 41] +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] + +(66) HashAggregate [codegen id : 41] +Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] +Keys [2]: [i_category#21, i_class#20] +Functions [1]: [sum(sumsales#129)] +Aggregate Attributes [1]: [sum(sumsales#129)#134] +Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#129)#134 AS sumsales#141] + +(67) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] + +(68) CometColumnarToRow [codegen id : 46] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] + +(69) HashAggregate [codegen id : 46] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] +Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] + +(70) HashAggregate [codegen id : 46] +Input [2]: [i_category#21, sumsales#149] +Keys [1]: [i_category#21] +Functions [1]: [partial_sum(sumsales#149)] +Aggregate Attributes [2]: [sum#150, isEmpty#151] +Results [3]: [i_category#21, sum#152, isEmpty#153] + +(71) CometColumnarExchange +Input [3]: [i_category#21, sum#152, isEmpty#153] +Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(72) CometColumnarToRow [codegen id : 47] +Input [3]: [i_category#21, sum#152, isEmpty#153] + +(73) HashAggregate [codegen id : 47] +Input [3]: [i_category#21, sum#152, isEmpty#153] +Keys [1]: [i_category#21] +Functions [1]: [sum(sumsales#149)] +Aggregate Attributes [1]: [sum(sumsales#149)#154] +Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i_product_name#157, null AS d_year#158, null AS d_qoy#159, null AS d_moy#160, null AS s_store_id#161, sum(sumsales#149)#154 AS sumsales#162] + +(74) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] + +(75) CometColumnarToRow [codegen id : 52] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] + +(76) HashAggregate [codegen id : 52] +Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] +Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] +Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] +Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] + +(77) HashAggregate [codegen id : 52] +Input [1]: [sumsales#170] Keys: [] -Functions [1]: [partial_sum(sumsales#103)] +Functions [1]: [partial_sum(sumsales#170)] +Aggregate Attributes [2]: [sum#171, isEmpty#172] +Results [2]: [sum#173, isEmpty#174] + +(78) CometColumnarExchange +Input [2]: [sum#173, isEmpty#174] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(62) CometExchange -Input [2]: [sum#104, isEmpty#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(79) CometColumnarToRow [codegen id : 53] +Input [2]: [sum#173, isEmpty#174] -(63) CometHashAggregate -Input [2]: [sum#104, isEmpty#105] +(80) HashAggregate [codegen id : 53] +Input [2]: [sum#173, isEmpty#174] Keys: [] -Functions [1]: [sum(sumsales#103)] +Functions [1]: [sum(sumsales#170)] +Aggregate Attributes [1]: [sum(sumsales#170)#175] +Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#170)#175 AS sumsales#184] + +(81) Union -(64) CometUnion -Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] -Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] -Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] +(82) Sort [codegen id : 54] +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 -(65) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] +(83) WindowGroupLimit +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +(84) CometColumnarExchange +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(67) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial +(85) CometSort +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] -(68) CometColumnarExchange -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(86) CometColumnarToRow [codegen id : 55] +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -(69) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] +(87) WindowGroupLimit +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +(88) Window +Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] -(71) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final +(89) Filter [codegen id : 56] +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +Condition : (rk#185 <= 100) -(72) Window -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] +(90) RowToColumnar +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -(73) Filter [codegen id : 3] -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Condition : (rk#159 <= 100) +(91) CometTakeOrderedAndProject +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#28 ASC NULLS FIRST,i_class#29 ASC NULLS FIRST,i_brand#30 ASC NULLS FIRST,i_product_name#31 ASC NULLS FIRST,d_year#32 ASC NULLS FIRST,d_qoy#33 ASC NULLS FIRST,d_moy#34 ASC NULLS FIRST,s_store_id#35 ASC NULLS FIRST,sumsales#36 ASC NULLS FIRST,rk#185 ASC NULLS FIRST], output=[i_category#28,i_class#29,i_brand#30,i_product_name#31,d_year#32,d_qoy#33,d_moy#34,s_store_id#35,sumsales#36,rk#185]), [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185], 100, 0, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#185 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -(74) TakeOrderedAndProject -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +(92) CometColumnarToRow [codegen id : 57] +Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (97) ++- * CometColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) +(94) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_month_seq#186 <= 1223)) AND isnotnull(d_date_sk#7)) -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(95) CometProject +Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +(96) CometColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(97) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt index da0a1c697a..275e74b65a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt @@ -1,298 +1,352 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- Sort + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 146 out of 285 eligible operators (51%). Final plan contains 65 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index 6096ee1c25..a56fc90d3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -1,88 +1,151 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +WholeStageCodegen (57) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + RowToColumnar + WholeStageCodegen (56) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (55) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (54) + Sort [i_category,sumsales] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (22) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (29) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (28) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (35) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (34) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (41) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #11 + WholeStageCodegen (40) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #12 + WholeStageCodegen (46) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #13 + WholeStageCodegen (52) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt index af71da5f87..82f75a1a91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt @@ -1,70 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) +* CometColumnarToRow (67) ++- CometTakeOrderedAndProject (66) + +- RowToColumnar (65) + +- * Project (64) + +- Window (63) + +- * CometColumnarToRow (62) + +- CometSort (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- * HashAggregate (57) + +- Union (56) + :- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * CometColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- WindowGroupLimit (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (25) + : +- CometColumnarExchange (24) + : +- * HashAggregate (23) + : +- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * CometColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : +- ReusedExchange (20) + :- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (55) + +- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- ReusedExchange (49) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -79,330 +80,333 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(3) CometColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#5] -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 8] +Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +(8) CometFilter +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Condition : isnotnull(s_store_sk#6) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] +(9) CometColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) +(11) CometFilter +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_store_sk#9) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] +(12) CometColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +(14) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : isnotnull(s_store_sk#13) -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] +(15) CometProject +Input [2]: [s_store_sk#13, s_state#14] +Arguments: [s_store_sk#13, s_state#15], [s_store_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) AS s_state#15] -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] +(16) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#13, s_state#15] -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final +(17) BroadcastExchange +Input [2]: [s_store_sk#13, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#9] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) +(19) Project [codegen id : 4] +Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15] +Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#15] -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +(20) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#16] -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +(21) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] +(22) Project [codegen id : 4] +Output [2]: [ss_net_profit#10, s_state#15] +Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15, d_date_sk#16] + +(23) HashAggregate [codegen id : 4] +Input [2]: [ss_net_profit#10, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum#17] +Results [2]: [s_state#15, sum#18] + +(24) CometColumnarExchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometColumnarToRow [codegen id : 5] +Input [2]: [s_state#15, sum#18] + +(26) HashAggregate [codegen id : 5] +Input [2]: [s_state#15, sum#18] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#19] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#19,17,2) AS _w0#20, s_state#15] + +(27) Sort [codegen id : 5] +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 + +(28) WindowGroupLimit +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final + +(29) Window +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] + +(30) Filter [codegen id : 6] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Condition : (ranking#21 <= 5) + +(31) Project [codegen id : 6] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] + +(32) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(33) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] +(34) Project [codegen id : 7] +Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#22] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(35) BroadcastExchange +Input [3]: [s_store_sk#6, s_county#7, s_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 4] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] +(37) Project [codegen id : 8] +Output [3]: [ss_net_profit#2, s_county#7, s_state#22] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#22] -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] +(38) HashAggregate [codegen id : 8] +Input [3]: [ss_net_profit#2, s_county#7, s_state#22] +Keys [2]: [s_state#22, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] +Aggregate Attributes [1]: [sum#23] +Results [3]: [s_state#22, s_county#7, sum#24] -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(39) CometColumnarExchange +Input [3]: [s_state#22, s_county#7, sum#24] +Arguments: hashpartitioning(s_state#22, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] +(40) CometColumnarToRow [codegen id : 9] +Input [3]: [s_state#22, s_county#7, sum#24] -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] +(41) HashAggregate [codegen id : 9] +Input [3]: [s_state#22, s_county#7, sum#24] +Keys [2]: [s_state#22, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#31, sum#32] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(47) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#45, sum#46] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#22 AS s_state#27, s_county#7 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] + +(42) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#22, s_county#32, sum#33] + +(43) CometColumnarToRow [codegen id : 18] +Input [3]: [s_state#22, s_county#32, sum#33] + +(44) HashAggregate [codegen id : 18] +Input [3]: [s_state#22, s_county#32, sum#33] +Keys [2]: [s_state#22, s_county#32] +Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#25] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#25,17,2) AS total_sum#35, s_state#22] + +(45) HashAggregate [codegen id : 18] +Input [2]: [total_sum#35, s_state#22] +Keys [1]: [s_state#22] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [3]: [s_state#22, sum#38, isEmpty#39] + +(46) CometColumnarExchange +Input [3]: [s_state#22, sum#38, isEmpty#39] +Arguments: hashpartitioning(s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(47) CometColumnarToRow [codegen id : 19] +Input [3]: [s_state#22, sum#38, isEmpty#39] + +(48) HashAggregate [codegen id : 19] +Input [3]: [s_state#22, sum#38, isEmpty#39] +Keys [1]: [s_state#22] +Functions [1]: [sum(total_sum#35)] +Aggregate Attributes [1]: [sum(total_sum#35)#40] +Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#22, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] + +(49) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#22, s_county#46, sum#47] + +(50) CometColumnarToRow [codegen id : 28] +Input [3]: [s_state#22, s_county#46, sum#47] + +(51) HashAggregate [codegen id : 28] +Input [3]: [s_state#22, s_county#46, sum#47] +Keys [2]: [s_state#22, s_county#46] +Functions [1]: [sum(UnscaledValue(ss_net_profit#48))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#48))#25] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#48))#25,17,2) AS total_sum#49] + +(52) HashAggregate [codegen id : 28] +Input [1]: [total_sum#49] Keys: [] -Functions [1]: [partial_sum(total_sum#48)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [2]: [sum#51, isEmpty#52] +Functions [1]: [partial_sum(total_sum#49)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [2]: [sum#52, isEmpty#53] -(54) CometColumnarExchange -Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(53) CometColumnarExchange +Input [2]: [sum#52, isEmpty#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] +(54) CometColumnarToRow [codegen id : 29] +Input [2]: [sum#52, isEmpty#53] -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] +(55) HashAggregate [codegen id : 29] +Input [2]: [sum#52, isEmpty#53] Keys: [] -Functions [1]: [sum(total_sum#48)] -Aggregate Attributes [1]: [sum(total_sum#48)#53] -Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] +Functions [1]: [sum(total_sum#49)] +Aggregate Attributes [1]: [sum(total_sum#49)#54] +Results [6]: [sum(total_sum#49)#54 AS total_sum#55, null AS s_state#56, null AS s_county#57, 1 AS g_state#58, 1 AS g_county#59, 2 AS lochierarchy#60] -(57) Union +(56) Union -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(57) HashAggregate [codegen id : 30] +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Results [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(58) CometColumnarExchange +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Arguments: hashpartitioning(total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(59) CometHashAggregate +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] Functions: [] -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(60) CometExchange +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: hashpartitioning(lochierarchy#31, _w0#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(61) CometSort +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61], [lochierarchy#31 ASC NULLS FIRST, _w0#61 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] + +(62) CometColumnarToRow [codegen id : 31] +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] +(63) Window +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#61, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#62], [lochierarchy#31, _w0#61], [total_sum#26 DESC NULLS LAST] -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +(64) Project [codegen id : 32] +Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61, rank_within_parent#62] -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] +(65) RowToColumnar +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] +(66) CometTakeOrderedAndProject +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#62 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#62]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#62 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +(67) CometColumnarToRow [codegen id : 33] +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) +BroadcastExchange (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(69) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#63] +Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +(70) CometProject +Input [2]: [d_date_sk#5, d_month_seq#63] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(71) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index 1574f09c56..dafedcf28b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -1,176 +1,190 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- 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 + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 156 eligible operators (42%). Final plan contains 32 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt index 2832c5a348..e8d5239ead 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt @@ -1,101 +1,111 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow +WholeStageCodegen (33) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (32) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (31) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (30) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (9) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (8) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (19) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #9 + WholeStageCodegen (18) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (29) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (28) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt index af71da5f87..82f75a1a91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt @@ -1,70 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) +* CometColumnarToRow (67) ++- CometTakeOrderedAndProject (66) + +- RowToColumnar (65) + +- * Project (64) + +- Window (63) + +- * CometColumnarToRow (62) + +- CometSort (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- * HashAggregate (57) + +- Union (56) + :- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * CometColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- WindowGroupLimit (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (25) + : +- CometColumnarExchange (24) + : +- * HashAggregate (23) + : +- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * CometColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (17) + : : +- * CometColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : +- ReusedExchange (20) + :- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (55) + +- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- ReusedExchange (49) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -79,330 +80,333 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(3) CometColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#5] -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 8] +Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#6, s_county#7, s_state#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) +(8) CometFilter +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Condition : isnotnull(s_store_sk#6) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] +(9) CometColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) +(11) CometFilter +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_store_sk#9) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] +(12) CometColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +(14) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : isnotnull(s_store_sk#13) -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] +(15) CometProject +Input [2]: [s_store_sk#13, s_state#14] +Arguments: [s_store_sk#13, s_state#15], [s_store_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) AS s_state#15] -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] +(16) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#13, s_state#15] -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final +(17) BroadcastExchange +Input [2]: [s_store_sk#13, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#9] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) +(19) Project [codegen id : 4] +Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15] +Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#15] -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +(20) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#16] -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +(21) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] +(22) Project [codegen id : 4] +Output [2]: [ss_net_profit#10, s_state#15] +Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15, d_date_sk#16] + +(23) HashAggregate [codegen id : 4] +Input [2]: [ss_net_profit#10, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum#17] +Results [2]: [s_state#15, sum#18] + +(24) CometColumnarExchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(25) CometColumnarToRow [codegen id : 5] +Input [2]: [s_state#15, sum#18] + +(26) HashAggregate [codegen id : 5] +Input [2]: [s_state#15, sum#18] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#19] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#19,17,2) AS _w0#20, s_state#15] + +(27) Sort [codegen id : 5] +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 + +(28) WindowGroupLimit +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final + +(29) Window +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] + +(30) Filter [codegen id : 6] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Condition : (ranking#21 <= 5) + +(31) Project [codegen id : 6] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] + +(32) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(33) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] +(34) Project [codegen id : 7] +Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#22] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(35) BroadcastExchange +Input [3]: [s_store_sk#6, s_county#7, s_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 4] +(36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] +(37) Project [codegen id : 8] +Output [3]: [ss_net_profit#2, s_county#7, s_state#22] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#22] -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] +(38) HashAggregate [codegen id : 8] +Input [3]: [ss_net_profit#2, s_county#7, s_state#22] +Keys [2]: [s_state#22, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] +Aggregate Attributes [1]: [sum#23] +Results [3]: [s_state#22, s_county#7, sum#24] -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(39) CometColumnarExchange +Input [3]: [s_state#22, s_county#7, sum#24] +Arguments: hashpartitioning(s_state#22, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] +(40) CometColumnarToRow [codegen id : 9] +Input [3]: [s_state#22, s_county#7, sum#24] -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] +(41) HashAggregate [codegen id : 9] +Input [3]: [s_state#22, s_county#7, sum#24] +Keys [2]: [s_state#22, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#31, sum#32] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(47) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#45, sum#46] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#22 AS s_state#27, s_county#7 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] + +(42) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#22, s_county#32, sum#33] + +(43) CometColumnarToRow [codegen id : 18] +Input [3]: [s_state#22, s_county#32, sum#33] + +(44) HashAggregate [codegen id : 18] +Input [3]: [s_state#22, s_county#32, sum#33] +Keys [2]: [s_state#22, s_county#32] +Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#25] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#25,17,2) AS total_sum#35, s_state#22] + +(45) HashAggregate [codegen id : 18] +Input [2]: [total_sum#35, s_state#22] +Keys [1]: [s_state#22] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [3]: [s_state#22, sum#38, isEmpty#39] + +(46) CometColumnarExchange +Input [3]: [s_state#22, sum#38, isEmpty#39] +Arguments: hashpartitioning(s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(47) CometColumnarToRow [codegen id : 19] +Input [3]: [s_state#22, sum#38, isEmpty#39] + +(48) HashAggregate [codegen id : 19] +Input [3]: [s_state#22, sum#38, isEmpty#39] +Keys [1]: [s_state#22] +Functions [1]: [sum(total_sum#35)] +Aggregate Attributes [1]: [sum(total_sum#35)#40] +Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#22, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] + +(49) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#22, s_county#46, sum#47] + +(50) CometColumnarToRow [codegen id : 28] +Input [3]: [s_state#22, s_county#46, sum#47] + +(51) HashAggregate [codegen id : 28] +Input [3]: [s_state#22, s_county#46, sum#47] +Keys [2]: [s_state#22, s_county#46] +Functions [1]: [sum(UnscaledValue(ss_net_profit#48))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#48))#25] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#48))#25,17,2) AS total_sum#49] + +(52) HashAggregate [codegen id : 28] +Input [1]: [total_sum#49] Keys: [] -Functions [1]: [partial_sum(total_sum#48)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [2]: [sum#51, isEmpty#52] +Functions [1]: [partial_sum(total_sum#49)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [2]: [sum#52, isEmpty#53] -(54) CometColumnarExchange -Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(53) CometColumnarExchange +Input [2]: [sum#52, isEmpty#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] +(54) CometColumnarToRow [codegen id : 29] +Input [2]: [sum#52, isEmpty#53] -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] +(55) HashAggregate [codegen id : 29] +Input [2]: [sum#52, isEmpty#53] Keys: [] -Functions [1]: [sum(total_sum#48)] -Aggregate Attributes [1]: [sum(total_sum#48)#53] -Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] +Functions [1]: [sum(total_sum#49)] +Aggregate Attributes [1]: [sum(total_sum#49)#54] +Results [6]: [sum(total_sum#49)#54 AS total_sum#55, null AS s_state#56, null AS s_county#57, 1 AS g_state#58, 1 AS g_county#59, 2 AS lochierarchy#60] -(57) Union +(56) Union -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(57) HashAggregate [codegen id : 30] +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Results [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(58) CometColumnarExchange +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Arguments: hashpartitioning(total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(59) CometHashAggregate +Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] Functions: [] -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(60) CometExchange +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: hashpartitioning(lochierarchy#31, _w0#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(61) CometSort +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61], [lochierarchy#31 ASC NULLS FIRST, _w0#61 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] + +(62) CometColumnarToRow [codegen id : 31] +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] +(63) Window +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#61, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#62], [lochierarchy#31, _w0#61], [total_sum#26 DESC NULLS LAST] -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +(64) Project [codegen id : 32] +Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61, rank_within_parent#62] -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] +(65) RowToColumnar +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] +(66) CometTakeOrderedAndProject +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#62 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#62]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#62 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +(67) CometColumnarToRow [codegen id : 33] +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) +BroadcastExchange (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(69) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#63] +Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +(70) CometProject +Input [2]: [d_date_sk#5, d_month_seq#63] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(71) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt index 1574f09c56..dafedcf28b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt @@ -1,176 +1,190 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- 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 + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 156 eligible operators (42%). Final plan contains 32 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index 2832c5a348..e8d5239ead 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -1,101 +1,111 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow +WholeStageCodegen (33) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (32) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (31) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (30) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (9) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (8) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (5) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (19) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #9 + WholeStageCodegen (18) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (29) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (28) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt index ee3242f2b1..49645a95cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt @@ -1,68 +1,74 @@ == Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) +* CometColumnarToRow (70) ++- CometTakeOrderedAndProject (69) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometSortMergeJoin (64) + :- CometSort (58) + : +- CometColumnarExchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * CometColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * CometColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * CometColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * CometColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (51) + +- CometSort (63) + +- CometExchange (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -77,7 +83,10 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(5) CometBroadcastExchange +(6) CometColumnarToRow [codegen id : 1] +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(7) BroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight +(8) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [inv_item_sk#10] +Join type: Inner +Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) -(7) CometProject +(9) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(9) CometFilter +(11) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(10) CometBroadcastExchange +(12) CometColumnarToRow [codegen id : 2] Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight +(13) BroadcastExchange +Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [inv_warehouse_sk#11] +Right keys [1]: [w_warehouse_sk#14] +Join type: Inner +Join condition: None -(12) CometProject +(15) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter +(17) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(15) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 3] Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight +(19) BroadcastExchange +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(17) CometProject +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [i_item_sk#16] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 10] +Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(23) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) -(20) CometProject +(24) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(21) CometBroadcastExchange +(25) CometColumnarToRow [codegen id : 4] +Input [1]: [cd_demo_sk#18] + +(26) BroadcastExchange Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight +(27) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#18] +Join type: Inner +Join condition: None -(23) CometProject +(28) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk)] ReadSchema: struct -(25) CometFilter +(30) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) -(26) CometProject +(31) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(27) CometBroadcastExchange +(32) CometColumnarToRow [codegen id : 5] Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct +(33) BroadcastExchange +Input [1]: [hd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: None -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] +(35) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] +(36) ReusedExchange [Reuses operator id: 75] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#8] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None -(35) CometProject +(38) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_week_seq#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) +(40) CometFilter +Input [2]: [d_date_sk#25, d_week_seq#26] +Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) + +(41) CometColumnarToRow [codegen id : 7] +Input [2]: [d_date_sk#25, d_week_seq#26] -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] +(42) BroadcastExchange +Input [2]: [d_date_sk#25, d_week_seq#26] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight +(43) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [d_week_seq#24, inv_date_sk#13] +Right keys [2]: [d_week_seq#26, d_date_sk#25] +Join type: Inner +Join condition: None -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(44) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_date#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) +(46) CometFilter +Input [2]: [d_date_sk#27, d_date#28] +Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] +(47) CometColumnarToRow [codegen id : 8] +Input [2]: [d_date_sk#27, d_date#28] -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight +(48) BroadcastExchange +Input [2]: [d_date_sk#27, d_date#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_ship_date_sk#1] +Right keys [1]: [d_date_sk#27] +Join type: Inner +Join condition: (d_date#28 > date_add(d_date#23, 5)) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] +(50) Project [codegen id : 10] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] + +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) +(52) CometFilter +Input [1]: [p_promo_sk#29] +Condition : isnotnull(p_promo_sk#29) + +(53) CometColumnarToRow [codegen id : 9] +Input [1]: [p_promo_sk#29] -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] +(54) BroadcastExchange +Input [1]: [p_promo_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight +(55) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_promo_sk#5] +Right keys [1]: [p_promo_sk#29] +Join type: LeftOuter +Join condition: None -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(56) Project [codegen id : 10] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(51) CometExchange +(57) CometColumnarExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(52) CometSort +(58) CometSort Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) +(60) CometFilter +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] +(61) CometProject +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(62) CometExchange +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] +(63) CometSort +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] -(58) CometSortMergeJoin +(64) CometSortMergeJoin Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter +Right output [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] +(65) CometProject +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(60) CometHashAggregate +(66) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(67) CometExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(68) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +(69) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +(70) CometColumnarToRow [codegen id : 11] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (75) ++- * CometColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(69) BroadcastExchange +(75) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index 9000d27d1b..f97ff81f45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- 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 - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- 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 + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 +Comet accelerated 39 out of 68 eligible operators (57%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt index 678f8c67df..3cdd634e67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] @@ -8,67 +8,100 @@ WholeStageCodegen (1) CometProject [w_warehouse_name,i_item_desc,d_week_seq] CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] - CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + CometColumnarToRow + InputAdapter + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_desc] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 + CometExchange [cr_item_sk,cr_order_number] #12 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt index ee3242f2b1..49645a95cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt @@ -1,68 +1,74 @@ == Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) +* CometColumnarToRow (70) ++- CometTakeOrderedAndProject (69) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometSortMergeJoin (64) + :- CometSort (58) + : +- CometColumnarExchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * CometColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * CometColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * CometColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * CometColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (51) + +- CometSort (63) + +- CometExchange (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -77,7 +83,10 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(5) CometBroadcastExchange +(6) CometColumnarToRow [codegen id : 1] +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(7) BroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight +(8) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [inv_item_sk#10] +Join type: Inner +Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) -(7) CometProject +(9) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(9) CometFilter +(11) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(10) CometBroadcastExchange +(12) CometColumnarToRow [codegen id : 2] Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight +(13) BroadcastExchange +Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [inv_warehouse_sk#11] +Right keys [1]: [w_warehouse_sk#14] +Join type: Inner +Join condition: None -(12) CometProject +(15) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter +(17) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(15) CometBroadcastExchange +(18) CometColumnarToRow [codegen id : 3] Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight +(19) BroadcastExchange +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(17) CometProject +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [i_item_sk#16] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 10] +Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(23) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) -(20) CometProject +(24) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(21) CometBroadcastExchange +(25) CometColumnarToRow [codegen id : 4] +Input [1]: [cd_demo_sk#18] + +(26) BroadcastExchange Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight +(27) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#18] +Join type: Inner +Join condition: None -(23) CometProject +(28) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk)] ReadSchema: struct -(25) CometFilter +(30) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) -(26) CometProject +(31) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(27) CometBroadcastExchange +(32) CometColumnarToRow [codegen id : 5] Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct +(33) BroadcastExchange +Input [1]: [hd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_bill_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: None -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] +(35) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] +(36) ReusedExchange [Reuses operator id: 75] +Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#8] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None -(35) CometProject +(38) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_week_seq#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) +(40) CometFilter +Input [2]: [d_date_sk#25, d_week_seq#26] +Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) + +(41) CometColumnarToRow [codegen id : 7] +Input [2]: [d_date_sk#25, d_week_seq#26] -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] +(42) BroadcastExchange +Input [2]: [d_date_sk#25, d_week_seq#26] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight +(43) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [d_week_seq#24, inv_date_sk#13] +Right keys [2]: [d_week_seq#26, d_date_sk#25] +Join type: Inner +Join condition: None -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(44) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_date#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) +(46) CometFilter +Input [2]: [d_date_sk#27, d_date#28] +Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] +(47) CometColumnarToRow [codegen id : 8] +Input [2]: [d_date_sk#27, d_date#28] -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight +(48) BroadcastExchange +Input [2]: [d_date_sk#27, d_date#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_ship_date_sk#1] +Right keys [1]: [d_date_sk#27] +Join type: Inner +Join condition: (d_date#28 > date_add(d_date#23, 5)) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] +(50) Project [codegen id : 10] +Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] + +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) +(52) CometFilter +Input [1]: [p_promo_sk#29] +Condition : isnotnull(p_promo_sk#29) + +(53) CometColumnarToRow [codegen id : 9] +Input [1]: [p_promo_sk#29] -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] +(54) BroadcastExchange +Input [1]: [p_promo_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight +(55) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_promo_sk#5] +Right keys [1]: [p_promo_sk#29] +Join type: LeftOuter +Join condition: None -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +(56) Project [codegen id : 10] +Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] -(51) CometExchange +(57) CometColumnarExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(52) CometSort +(58) CometSort Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) +(60) CometFilter +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] +(61) CometProject +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(62) CometExchange +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] +(63) CometSort +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] -(58) CometSortMergeJoin +(64) CometSortMergeJoin Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter +Right output [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] +(65) CometProject +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(60) CometHashAggregate +(66) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(67) CometExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(68) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +(69) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +(70) CometColumnarToRow [codegen id : 11] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (75) ++- * CometColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(74) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(69) BroadcastExchange +(75) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt index 9000d27d1b..f97ff81f45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- 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 - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- 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 + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- CometColumnarToRow + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 +Comet accelerated 39 out of 68 eligible operators (57%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index 678f8c67df..3cdd634e67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] @@ -8,67 +8,100 @@ WholeStageCodegen (1) CometProject [w_warehouse_name,i_item_desc,d_week_seq] CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] - CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] - CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] - CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (10) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + CometColumnarToRow + InputAdapter + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_desc] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 + CometExchange [cr_item_sk,cr_order_number] #12 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt index b0ae8bb695..d3008194f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt @@ -1,76 +1,85 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) +* CometColumnarToRow (81) ++- CometTakeOrderedAndProject (80) + +- RowToColumnar (79) + +- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * CometColumnarToRow (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (26) + : : : +- * CometColumnarToRow (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : +- ReusedExchange (29) + : +- BroadcastExchange (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometFilter (39) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (48) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * CometColumnarToRow (74) + +- CometColumnarExchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * CometColumnarToRow (62) + : : +- CometProject (61) + : : +- CometFilter (60) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometFilter (64) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (69) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -88,7 +97,10 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 3] +Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] @@ -96,375 +108,414 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Condition : isnotnull(ss_customer_sk#8) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] + +(8) BroadcastExchange Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#8] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -(14) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] +Aggregate Attributes [1]: [sum#14] +Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(15) CometColumnarExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] +(16) CometColumnarToRow [codegen id : 16] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] + +(17) HashAggregate [codegen id : 16] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] +Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) +(18) Filter [codegen id : 16] +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) +(20) CometFilter +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) + +(21) CometProject +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] +(22) CometColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(24) CometFilter +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) + +(25) CometColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] + +(26) BroadcastExchange +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] + +(29) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#27, d_year#28] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#27] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 6] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] +Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] + +(32) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#29] +Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] + +(33) CometColumnarExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometColumnarToRow [codegen id : 7] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] + +(35) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] +Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] + +(36) BroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#31] +Join type: Inner +Join condition: None + +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) +(39) CometFilter +Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] +(40) CometProject +Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +(41) CometColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight +(43) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] +(44) CometColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +(45) BroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#35] +Right keys [1]: [ws_bill_customer_sk#42] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 10] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] + +(48) ReusedExchange [Reuses operator id: 85] +Output [2]: [d_date_sk#46, d_year#47] + +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#46] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 10] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] +Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] + +(51) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#48] +Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] + +(52) CometColumnarExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometColumnarToRow [codegen id : 11] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] + +(54) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#50] +Results [2]: [c_customer_id#39 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#50,17,2) AS year_total#52] + +(55) Filter [codegen id : 11] +Input [2]: [customer_id#51, year_total#52] +Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) + +(56) BroadcastExchange +Input [2]: [customer_id#51, year_total#52] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#51] +Join type: Inner +Join condition: None + +(58) Project [codegen id : 16] +Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52] +Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#51, year_total#52] + +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) +(60) CometFilter +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Condition : (isnotnull(c_customer_sk#53) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true))) -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] +(61) CometProject +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Arguments: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#55, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#56, 30, true, false, true) AS c_last_name#41] + +(62) CometColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] +(64) CometFilter +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Condition : isnotnull(ws_bill_customer_sk#57) + +(65) CometColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] + +(66) BroadcastExchange +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#53] +Right keys [1]: [ws_bill_customer_sk#57] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59] +Input [7]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] + +(69) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#61, d_year#62] + +(70) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#59] +Right keys [1]: [d_date_sk#61] +Join type: Inner +Join condition: None + +(71) Project [codegen id : 14] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] +Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59, d_date_sk#61, d_year#62] + +(72) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#58))] +Aggregate Attributes [1]: [sum#63] +Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] + +(73) CometColumnarExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometColumnarToRow [codegen id : 15] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] + +(75) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] +Functions [1]: [sum(UnscaledValue(ws_net_paid#58))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#58))#50] +Results [2]: [c_customer_id#39 AS customer_id#65, MakeDecimal(sum(UnscaledValue(ws_net_paid#58))#50,17,2) AS year_total#66] -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] +(76) BroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#65] +Join type: Inner +Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#66 / year_total#52) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight +(78) Project [codegen id : 16] +Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52, customer_id#65, year_total#66] -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] +(79) RowToColumnar +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +(80) CometTakeOrderedAndProject +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +(81) CometColumnarToRow [codegen id : 17] +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometFilter (83) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(83) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(76) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_year#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +(87) CometFilter +Input [2]: [d_date_sk#27, d_year#28] +Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] +(88) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#27, d_year#28] -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(89) BroadcastExchange +Input [2]: [d_date_sk#27, d_year#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#26 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index 476c7be954..f5dda0de66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -1,90 +1,107 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt index 72e39422e0..cc6790a5db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt @@ -1,90 +1,134 @@ -WholeStageCodegen (1) +WholeStageCodegen (17) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + RowToColumnar + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year] #4 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year] #9 + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt index b0ae8bb695..d3008194f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt @@ -1,76 +1,85 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) +* CometColumnarToRow (81) ++- CometTakeOrderedAndProject (80) + +- RowToColumnar (79) + +- * Project (78) + +- * BroadcastHashJoin Inner BuildRight (77) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * Project (28) + : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : :- * CometColumnarToRow (22) + : : : : +- CometProject (21) + : : : : +- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (26) + : : : +- * CometColumnarToRow (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : +- ReusedExchange (29) + : +- BroadcastExchange (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- CometColumnarExchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometFilter (39) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) + : : +- BroadcastExchange (45) + : : +- * CometColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : +- ReusedExchange (48) + +- BroadcastExchange (76) + +- * HashAggregate (75) + +- * CometColumnarToRow (74) + +- CometColumnarExchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (68) + : +- * BroadcastHashJoin Inner BuildRight (67) + : :- * CometColumnarToRow (62) + : : +- CometProject (61) + : : +- CometFilter (60) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) + : +- BroadcastExchange (66) + : +- * CometColumnarToRow (65) + : +- CometFilter (64) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) + +- ReusedExchange (69) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -88,7 +97,10 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometColumnarToRow [codegen id : 3] +Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] + +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] @@ -96,375 +108,414 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(5) CometFilter +(6) CometFilter Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Condition : isnotnull(ss_customer_sk#8) -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] + +(8) BroadcastExchange Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#8] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 85] Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -(14) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] +Aggregate Attributes [1]: [sum#14] +Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(15) CometColumnarExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] +(16) CometColumnarToRow [codegen id : 16] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] + +(17) HashAggregate [codegen id : 16] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] +Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) +(18) Filter [codegen id : 16] +Input [2]: [customer_id#17, year_total#18] +Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) +(20) CometFilter +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) + +(21) CometProject +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] +(22) CometColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(24) CometFilter +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) + +(25) CometColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] + +(26) BroadcastExchange +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] + +(29) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#27, d_year#28] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#27] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 6] +Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] +Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] + +(32) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#29] +Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] + +(33) CometColumnarExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(34) CometColumnarToRow [codegen id : 7] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] + +(35) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] +Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] + +(36) BroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#31] +Join type: Inner +Join condition: None + +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) +(39) CometFilter +Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] +(40) CometProject +Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +(41) CometColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight +(43) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] +(44) CometColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +(45) BroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#35] +Right keys [1]: [ws_bill_customer_sk#42] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 10] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] + +(48) ReusedExchange [Reuses operator id: 85] +Output [2]: [d_date_sk#46, d_year#47] + +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#46] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 10] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] +Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] + +(51) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#48] +Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] + +(52) CometColumnarExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(53) CometColumnarToRow [codegen id : 11] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] + +(54) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#50] +Results [2]: [c_customer_id#39 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#50,17,2) AS year_total#52] + +(55) Filter [codegen id : 11] +Input [2]: [customer_id#51, year_total#52] +Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) + +(56) BroadcastExchange +Input [2]: [customer_id#51, year_total#52] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(57) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#51] +Join type: Inner +Join condition: None + +(58) Project [codegen id : 16] +Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52] +Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#51, year_total#52] + +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) +(60) CometFilter +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Condition : (isnotnull(c_customer_sk#53) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true))) -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] +(61) CometProject +Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +Arguments: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#55, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#56, 30, true, false, true) AS c_last_name#41] + +(62) CometColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] +(64) CometFilter +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Condition : isnotnull(ws_bill_customer_sk#57) + +(65) CometColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] + +(66) BroadcastExchange +Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(67) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#53] +Right keys [1]: [ws_bill_customer_sk#57] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 14] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59] +Input [7]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] + +(69) ReusedExchange [Reuses operator id: 89] +Output [2]: [d_date_sk#61, d_year#62] + +(70) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#59] +Right keys [1]: [d_date_sk#61] +Join type: Inner +Join condition: None + +(71) Project [codegen id : 14] +Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] +Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59, d_date_sk#61, d_year#62] + +(72) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#58))] +Aggregate Attributes [1]: [sum#63] +Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] + +(73) CometColumnarExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(74) CometColumnarToRow [codegen id : 15] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] + +(75) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] +Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] +Functions [1]: [sum(UnscaledValue(ws_net_paid#58))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#58))#50] +Results [2]: [c_customer_id#39 AS customer_id#65, MakeDecimal(sum(UnscaledValue(ws_net_paid#58))#50,17,2) AS year_total#66] -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] +(76) BroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] +(77) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#17] +Right keys [1]: [customer_id#65] +Join type: Inner +Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#66 / year_total#52) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight +(78) Project [codegen id : 16] +Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52, customer_id#65, year_total#66] -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] +(79) RowToColumnar +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +(80) CometTakeOrderedAndProject +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +(81) CometColumnarToRow [codegen id : 17] +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (85) ++- * CometColumnarToRow (84) + +- CometFilter (83) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(83) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(84) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(76) BroadcastExchange +(85) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_year#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +(87) CometFilter +Input [2]: [d_date_sk#27, d_year#28] +Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] +(88) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#27, d_year#28] -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(89) BroadcastExchange +Input [2]: [d_date_sk#27, d_year#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#26 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt index 476c7be954..f5dda0de66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt @@ -1,90 +1,107 @@ CometColumnarToRow +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 + +- RowToColumnar + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index 72e39422e0..cc6790a5db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -1,90 +1,134 @@ -WholeStageCodegen (1) +WholeStageCodegen (17) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow + RowToColumnar + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year] #4 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year] #9 + BroadcastExchange #10 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt index 6925eb301e..b48350e78c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt @@ -1,128 +1,131 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) +* CometColumnarToRow (127) ++- CometTakeOrderedAndProject (126) + +- CometProject (125) + +- CometSortMergeJoin (124) + :- CometSort (68) + : +- CometExchange (67) + : +- CometFilter (66) + : +- CometHashAggregate (65) + : +- CometExchange (64) + : +- CometHashAggregate (63) + : +- CometHashAggregate (62) + : +- CometExchange (61) + : +- CometHashAggregate (60) + : +- CometUnion (59) : :- CometProject (22) : : +- CometSortMergeJoin (21) : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : : +- CometColumnarExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) : : +- CometSort (20) : : +- CometExchange (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) + : :- CometProject (40) + : : +- CometSortMergeJoin (39) + : : :- CometSort (33) + : : : +- CometColumnarExchange (32) + : : : +- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * CometColumnarToRow (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (26) + : : : +- ReusedExchange (29) + : : +- CometSort (38) + : : +- CometExchange (37) + : : +- CometProject (36) + : : +- CometFilter (35) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (34) + : +- CometProject (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometColumnarExchange (50) + : : +- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * CometColumnarToRow (43) + : : : : +- CometFilter (42) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) + : : : +- ReusedExchange (44) + : : +- ReusedExchange (47) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (52) + +- CometSort (123) + +- CometExchange (122) + +- CometFilter (121) + +- CometHashAggregate (120) + +- CometExchange (119) + +- CometHashAggregate (118) + +- CometHashAggregate (117) + +- CometExchange (116) + +- CometHashAggregate (115) + +- CometUnion (114) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * CometColumnarToRow (71) + : : : : +- CometFilter (70) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (98) + : +- CometSortMergeJoin (97) + : :- CometSort (94) + : : +- CometColumnarExchange (93) + : : +- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (89) + : : : +- * BroadcastHashJoin Inner BuildRight (88) + : : : :- * CometColumnarToRow (86) + : : : : +- CometFilter (85) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (90) + : +- CometSort (96) + : +- ReusedExchange (95) + +- CometProject (113) + +- CometSortMergeJoin (112) + :- CometSort (109) + : +- CometColumnarExchange (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Project (104) + : : +- * BroadcastHashJoin Inner BuildRight (103) + : : :- * CometColumnarToRow (101) + : : : +- CometFilter (100) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (99) + : : +- ReusedExchange (102) + : +- ReusedExchange (105) + +- CometSort (111) + +- ReusedExchange (110) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -137,61 +140,57 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] + +(8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) CometExchange +(14) CometColumnarExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (15) CometSort Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] @@ -214,7 +213,7 @@ Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_ (19) CometExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (20) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -241,71 +240,76 @@ ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(35) CometProject +(36) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(36) CometExchange +(37) CometExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(37) CometSort +(38) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(38) CometSortMergeJoin +(39) CometSortMergeJoin Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(39) CometProject +(40) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -313,120 +317,125 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(41) CometFilter +(42) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(42) ReusedExchange [Reuses operator id: 6] +(43) CometColumnarToRow [codegen id : 9] +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] + +(44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight +(45) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#42] +Right keys [1]: [i_item_sk#48] +Join type: Inner +Join condition: None -(44) CometProject +(46) Project [codegen id : 9] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(45) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#53, d_year#54] -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight +(48) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(47) CometProject +(49) Project [codegen id : 9] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(48) CometExchange +(50) CometColumnarExchange Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) CometSort +(51) CometSort Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(51) CometFilter +(53) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(52) CometProject +(54) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(53) CometExchange +(55) CometExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(54) CometSort +(56) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(55) CometSortMergeJoin +(57) CometSortMergeJoin Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(56) CometProject +(58) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(57) CometUnion +(59) CometUnion Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(58) CometHashAggregate +(60) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(59) CometExchange +(61) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(60) CometHashAggregate +(62) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) CometHashAggregate +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(62) CometExchange +(64) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(63) CometHashAggregate +(65) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(64) CometFilter +(66) CometFilter Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Condition : isnotnull(sales_cnt#64) -(65) CometExchange +(67) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(66) CometSort +(68) CometSort Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] @@ -434,71 +443,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter +(70) CometFilter Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Condition : isnotnull(cs_item_sk#66) -(69) ReusedExchange [Reuses operator id: 6] +(71) CometColumnarToRow [codegen id : 12] +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] + +(72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight +(73) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [cs_item_sk#66] +Right keys [1]: [i_item_sk#72] +Join type: Inner +Join condition: None -(71) CometProject +(74) Project [codegen id : 12] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight +(76) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#77] +Join type: Inner +Join condition: None -(76) CometProject +(77) Project [codegen id : 12] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(77) CometExchange +(78) CometColumnarExchange Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(78) CometSort +(79) CometSort Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(79) ReusedExchange [Reuses operator id: 19] +(80) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(80) CometSort +(81) CometSort Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(81) CometSortMergeJoin +(82) CometSortMergeJoin Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(82) CometProject +(83) CometProject Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -506,59 +508,64 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(84) CometFilter +(85) CometFilter Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) -(85) ReusedExchange [Reuses operator id: 6] +(86) CometColumnarToRow [codegen id : 15] +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] + +(87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight +(88) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ss_item_sk#83] +Right keys [1]: [i_item_sk#89] +Join type: Inner +Join condition: None -(87) CometProject +(89) Project [codegen id : 15] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(88) ReusedExchange [Reuses operator id: 74] +(90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#94, d_year#95] -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight +(91) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None -(90) CometProject +(92) Project [codegen id : 15] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(91) CometExchange +(93) CometColumnarExchange Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(92) CometSort +(94) CometSort Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(93) ReusedExchange [Reuses operator id: 36] +(95) ReusedExchange [Reuses operator id: 37] Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(94) CometSort +(96) CometSort Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(95) CometSortMergeJoin +(97) CometSortMergeJoin Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(96) CometProject +(98) CometProject Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] @@ -566,177 +573,182 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(98) CometFilter +(100) CometFilter Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Condition : isnotnull(ws_item_sk#100) -(99) ReusedExchange [Reuses operator id: 6] +(101) CometColumnarToRow [codegen id : 18] +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] + +(102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight +(103) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_item_sk#100] +Right keys [1]: [i_item_sk#106] +Join type: Inner +Join condition: None -(101) CometProject +(104) Project [codegen id : 18] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(102) ReusedExchange [Reuses operator id: 74] +(105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#111, d_year#112] -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight +(106) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#104] +Right keys [1]: [d_date_sk#111] +Join type: Inner +Join condition: None -(104) CometProject +(107) Project [codegen id : 18] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(105) CometExchange +(108) CometColumnarExchange Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(106) CometSort +(109) CometSort Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(107) ReusedExchange [Reuses operator id: 53] +(110) ReusedExchange [Reuses operator id: 55] Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(108) CometSort +(111) CometSort Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(109) CometSortMergeJoin +(112) CometSortMergeJoin Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(110) CometProject +(113) CometProject Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(111) CometUnion +(114) CometUnion Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(112) CometHashAggregate +(115) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(113) CometExchange +(116) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] -(114) CometHashAggregate +(117) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(115) CometHashAggregate +(118) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(116) CometExchange +(119) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(117) CometHashAggregate +(120) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(118) CometFilter +(121) CometFilter Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Condition : isnotnull(sales_cnt#118) -(119) CometExchange +(122) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] -(120) CometSort +(123) CometSort Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(121) CometSortMergeJoin +(124) CometSortMergeJoin Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(122) CometProject +(125) CometProject Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -(123) CometTakeOrderedAndProject +(126) CometTakeOrderedAndProject Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 19] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (131) ++- * CometColumnarToRow (130) + +- CometFilter (129) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(129) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(130) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(128) BroadcastExchange +(131) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) +Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (135) ++- * CometColumnarToRow (134) + +- CometFilter (133) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (132) -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(130) CometFilter +(133) CometFilter Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(134) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] -(132) BroadcastExchange +(135) BroadcastExchange Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 22ccc17641..0765ce2338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- 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 + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- 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 + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 +Comet accelerated 123 out of 167 eligible operators (73%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt index d0520c8b63..2f283a3eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (19) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -17,128 +17,157 @@ WholeStageCodegen (1) CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [cs_order_number,cs_item_sk] #4 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 + CometExchange [cr_order_number,cr_item_sk] #7 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ss_ticket_number,ss_item_sk] #9 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #7 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (6) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 + CometExchange [sr_ticket_number,sr_item_sk] #9 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ws_order_number,ws_item_sk] #11 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #7 + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (9) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 + CometExchange [wr_order_number,wr_item_sk] #11 CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [cs_order_number,cs_item_sk] #16 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (12) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ss_ticket_number,ss_item_sk] #19 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #18 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (15) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ws_order_number,ws_item_sk] #20 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #18 + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (18) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt index 6925eb301e..b48350e78c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt @@ -1,128 +1,131 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) +* CometColumnarToRow (127) ++- CometTakeOrderedAndProject (126) + +- CometProject (125) + +- CometSortMergeJoin (124) + :- CometSort (68) + : +- CometExchange (67) + : +- CometFilter (66) + : +- CometHashAggregate (65) + : +- CometExchange (64) + : +- CometHashAggregate (63) + : +- CometHashAggregate (62) + : +- CometExchange (61) + : +- CometHashAggregate (60) + : +- CometUnion (59) : :- CometProject (22) : : +- CometSortMergeJoin (21) : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : : +- CometColumnarExchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * CometColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) : : +- CometSort (20) : : +- CometExchange (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) + : :- CometProject (40) + : : +- CometSortMergeJoin (39) + : : :- CometSort (33) + : : : +- CometColumnarExchange (32) + : : : +- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * CometColumnarToRow (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (26) + : : : +- ReusedExchange (29) + : : +- CometSort (38) + : : +- CometExchange (37) + : : +- CometProject (36) + : : +- CometFilter (35) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (34) + : +- CometProject (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometColumnarExchange (50) + : : +- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * Project (46) + : : : +- * BroadcastHashJoin Inner BuildRight (45) + : : : :- * CometColumnarToRow (43) + : : : : +- CometFilter (42) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) + : : : +- ReusedExchange (44) + : : +- ReusedExchange (47) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (52) + +- CometSort (123) + +- CometExchange (122) + +- CometFilter (121) + +- CometHashAggregate (120) + +- CometExchange (119) + +- CometHashAggregate (118) + +- CometHashAggregate (117) + +- CometExchange (116) + +- CometHashAggregate (115) + +- CometUnion (114) + :- CometProject (83) + : +- CometSortMergeJoin (82) + : :- CometSort (79) + : : +- CometColumnarExchange (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * CometColumnarToRow (71) + : : : : +- CometFilter (70) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- CometSort (81) + : +- ReusedExchange (80) + :- CometProject (98) + : +- CometSortMergeJoin (97) + : :- CometSort (94) + : : +- CometColumnarExchange (93) + : : +- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (89) + : : : +- * BroadcastHashJoin Inner BuildRight (88) + : : : :- * CometColumnarToRow (86) + : : : : +- CometFilter (85) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (90) + : +- CometSort (96) + : +- ReusedExchange (95) + +- CometProject (113) + +- CometSortMergeJoin (112) + :- CometSort (109) + : +- CometColumnarExchange (108) + : +- * Project (107) + : +- * BroadcastHashJoin Inner BuildRight (106) + : :- * Project (104) + : : +- * BroadcastHashJoin Inner BuildRight (103) + : : :- * CometColumnarToRow (101) + : : : +- CometFilter (100) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (99) + : : +- ReusedExchange (102) + : +- ReusedExchange (105) + +- CometSort (111) + +- ReusedExchange (110) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -137,61 +140,57 @@ ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] +Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] + +(8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None -(8) CometProject +(10) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None -(13) CometProject +(13) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) CometExchange +(14) CometColumnarExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] (15) CometSort Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] @@ -214,7 +213,7 @@ Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_ (19) CometExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (20) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -241,71 +240,76 @@ ReadSchema: struct -(34) CometFilter +(35) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(35) CometProject +(36) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(36) CometExchange +(37) CometExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(37) CometSort +(38) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(38) CometSortMergeJoin +(39) CometSortMergeJoin Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(39) CometProject +(40) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -313,120 +317,125 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(41) CometFilter +(42) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(42) ReusedExchange [Reuses operator id: 6] +(43) CometColumnarToRow [codegen id : 9] +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] + +(44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight +(45) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#42] +Right keys [1]: [i_item_sk#48] +Join type: Inner +Join condition: None -(44) CometProject +(46) Project [codegen id : 9] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(45) ReusedExchange [Reuses operator id: 11] +(47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#53, d_year#54] -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight +(48) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(47) CometProject +(49) Project [codegen id : 9] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(48) CometExchange +(50) CometColumnarExchange Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) CometSort +(51) CometSort Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(51) CometFilter +(53) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(52) CometProject +(54) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(53) CometExchange +(55) CometExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(54) CometSort +(56) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(55) CometSortMergeJoin +(57) CometSortMergeJoin Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(56) CometProject +(58) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(57) CometUnion +(59) CometUnion Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(58) CometHashAggregate +(60) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(59) CometExchange +(61) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(60) CometHashAggregate +(62) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) CometHashAggregate +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(62) CometExchange +(64) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(63) CometHashAggregate +(65) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(64) CometFilter +(66) CometFilter Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Condition : isnotnull(sales_cnt#64) -(65) CometExchange +(67) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(66) CometSort +(68) CometSort Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] @@ -434,71 +443,64 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter +(70) CometFilter Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Condition : isnotnull(cs_item_sk#66) -(69) ReusedExchange [Reuses operator id: 6] +(71) CometColumnarToRow [codegen id : 12] +Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] + +(72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight +(73) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [cs_item_sk#66] +Right keys [1]: [i_item_sk#72] +Join type: Inner +Join condition: None -(71) CometProject +(74) Project [codegen id : 12] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight +(76) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [cs_sold_date_sk#70] +Right keys [1]: [d_date_sk#77] +Join type: Inner +Join condition: None -(76) CometProject +(77) Project [codegen id : 12] +Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(77) CometExchange +(78) CometColumnarExchange Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(78) CometSort +(79) CometSort Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(79) ReusedExchange [Reuses operator id: 19] +(80) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(80) CometSort +(81) CometSort Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(81) CometSortMergeJoin +(82) CometSortMergeJoin Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(82) CometProject +(83) CometProject Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -506,59 +508,64 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(84) CometFilter +(85) CometFilter Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) -(85) ReusedExchange [Reuses operator id: 6] +(86) CometColumnarToRow [codegen id : 15] +Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] + +(87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight +(88) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ss_item_sk#83] +Right keys [1]: [i_item_sk#89] +Join type: Inner +Join condition: None -(87) CometProject +(89) Project [codegen id : 15] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(88) ReusedExchange [Reuses operator id: 74] +(90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#94, d_year#95] -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight +(91) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#94] +Join type: Inner +Join condition: None -(90) CometProject +(92) Project [codegen id : 15] +Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(91) CometExchange +(93) CometColumnarExchange Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(92) CometSort +(94) CometSort Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(93) ReusedExchange [Reuses operator id: 36] +(95) ReusedExchange [Reuses operator id: 37] Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(94) CometSort +(96) CometSort Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(95) CometSortMergeJoin +(97) CometSortMergeJoin Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(96) CometProject +(98) CometProject Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] @@ -566,177 +573,182 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(98) CometFilter +(100) CometFilter Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Condition : isnotnull(ws_item_sk#100) -(99) ReusedExchange [Reuses operator id: 6] +(101) CometColumnarToRow [codegen id : 18] +Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] + +(102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight +(103) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_item_sk#100] +Right keys [1]: [i_item_sk#106] +Join type: Inner +Join condition: None -(101) CometProject +(104) Project [codegen id : 18] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(102) ReusedExchange [Reuses operator id: 74] +(105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#111, d_year#112] -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight +(106) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#104] +Right keys [1]: [d_date_sk#111] +Join type: Inner +Join condition: None -(104) CometProject +(107) Project [codegen id : 18] +Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(105) CometExchange +(108) CometColumnarExchange Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(106) CometSort +(109) CometSort Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(107) ReusedExchange [Reuses operator id: 53] +(110) ReusedExchange [Reuses operator id: 55] Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(108) CometSort +(111) CometSort Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(109) CometSortMergeJoin +(112) CometSortMergeJoin Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(110) CometProject +(113) CometProject Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(111) CometUnion +(114) CometUnion Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(112) CometHashAggregate +(115) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(113) CometExchange +(116) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] -(114) CometHashAggregate +(117) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(115) CometHashAggregate +(118) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(116) CometExchange +(119) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(117) CometHashAggregate +(120) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(118) CometFilter +(121) CometFilter Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Condition : isnotnull(sales_cnt#118) -(119) CometExchange +(122) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] -(120) CometSort +(123) CometSort Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(121) CometSortMergeJoin +(124) CometSortMergeJoin Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(122) CometProject +(125) CometProject Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -(123) CometTakeOrderedAndProject +(126) CometTakeOrderedAndProject Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(124) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 19] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (131) ++- * CometColumnarToRow (130) + +- CometFilter (129) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(129) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(130) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(128) BroadcastExchange +(131) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) +Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (135) ++- * CometColumnarToRow (134) + +- CometFilter (133) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (132) -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(130) CometFilter +(133) CometFilter Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(134) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] -(132) BroadcastExchange +(135) BroadcastExchange Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt index 22ccc17641..0765ce2338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- 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 + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- 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 + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- 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 + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 +Comet accelerated 123 out of 167 eligible operators (73%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index d0520c8b63..2f283a3eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (1) +WholeStageCodegen (19) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -17,128 +17,157 @@ WholeStageCodegen (1) CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [cs_order_number,cs_item_sk] #4 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 + CometExchange [cr_order_number,cr_item_sk] #7 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ss_ticket_number,ss_item_sk] #9 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #7 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (6) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 + CometExchange [sr_ticket_number,sr_item_sk] #9 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ws_order_number,ws_item_sk] #11 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #7 + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (9) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 + CometExchange [wr_order_number,wr_item_sk] #11 CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [cs_order_number,cs_item_sk] #16 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - CometBroadcastExchange [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (12) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ss_ticket_number,ss_item_sk] #19 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #18 + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (15) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometExchange [ws_order_number,ws_item_sk] #20 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - ReusedExchange [d_date_sk,d_year] #18 + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (18) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt index 4b5efc84e3..5f6ee267fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt @@ -1,107 +1,114 @@ == Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) +* CometColumnarToRow (110) ++- CometTakeOrderedAndProject (109) + +- CometHashAggregate (108) + +- CometColumnarExchange (107) + +- * HashAggregate (106) + +- Union (105) :- * HashAggregate (90) : +- * CometColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) + : +- Union (86) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : : :- * HashAggregate (16) + : : : +- * CometColumnarToRow (15) + : : : +- CometColumnarExchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * CometColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * CometColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : :- * Project (53) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) + : : :- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- * CometColumnarToRow (40) + : : : +- CometColumnarExchange (39) + : : : +- * HashAggregate (38) + : : : +- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * CometColumnarToRow (34) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) + : : : +- ReusedExchange (35) + : : +- * HashAggregate (51) + : : +- * CometColumnarToRow (50) + : : +- CometColumnarExchange (49) + : : +- * HashAggregate (48) + : : +- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : +- ReusedExchange (45) + : +- * Project (85) + : +- * BroadcastHashJoin LeftOuter BuildRight (84) + : :- * HashAggregate (69) + : : +- * CometColumnarToRow (68) + : : +- CometColumnarExchange (67) + : : +- * HashAggregate (66) + : : +- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * CometColumnarToRow (56) + : : : : +- CometFilter (55) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (54) + : : : +- ReusedExchange (57) + : : +- BroadcastExchange (63) + : : +- * CometColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (60) + : +- BroadcastExchange (83) + : +- * HashAggregate (82) + : +- * CometColumnarToRow (81) + : +- CometColumnarExchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * CometColumnarToRow (72) + : : : +- CometFilter (71) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (70) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + :- * HashAggregate (97) + : +- * CometColumnarToRow (96) + : +- CometColumnarExchange (95) + : +- * HashAggregate (94) + : +- * HashAggregate (93) + : +- * CometColumnarToRow (92) + : +- ReusedExchange (91) + +- * HashAggregate (104) + +- * CometColumnarToRow (103) + +- CometColumnarExchange (102) + +- * HashAggregate (101) + +- * HashAggregate (100) + +- * CometColumnarToRow (99) + +- ReusedExchange (98) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,506 +123,556 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] +(3) CometColumnarToRow [codegen id : 3] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] +(4) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#6] -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 3] +Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [1]: [s_store_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) +(8) CometFilter +Input [1]: [s_store_sk#7] +Condition : isnotnull(s_store_sk#7) + +(9) CometColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#7] -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] +(10) BroadcastExchange +Input [1]: [s_store_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +(12) Project [codegen id : 3] +Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] +(13) HashAggregate [codegen id : 3] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#8, sum#9] +Results [3]: [s_store_sk#7, sum#10, sum#11] -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(14) CometColumnarExchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] +(15) CometColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] + +(16) HashAggregate [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] +Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight +(19) CometColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +(20) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#21] -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] +(22) Project [codegen id : 6] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] +(23) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#22] -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] +(25) Project [codegen id : 6] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] + +(26) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] + +(27) CometColumnarExchange +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(28) CometColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] + +(29) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] + +(30) BroadcastExchange +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#22] +Join type: LeftOuter +Join condition: None -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +(32) Project [codegen id : 8] +Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] +(34) CometColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(35) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#40] -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] +(36) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +(37) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] + +(38) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(39) CometColumnarExchange +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) CometColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(41) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] + +(42) BroadcastExchange +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] +(44) CometColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight +(45) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#53] -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] +(46) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] +(47) Project [codegen id : 13] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] + +(48) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(49) CometColumnarExchange +Input [2]: [sum#56, sum#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] +(50) CometColumnarToRow +Input [2]: [sum#56, sum#57] -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] +(51) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] -(49) BroadcastNestedLoopJoin [codegen id : 3] +(52) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] +(53) Project [codegen id : 14] +Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) +(55) CometFilter +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) + +(56) CometColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] +(57) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#70] -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] +Join type: Inner +Join condition: None -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +(59) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) +(61) CometFilter +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] +(62) CometColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#71] -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight +(63) BroadcastExchange +Input [1]: [wp_web_page_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] +Join type: Inner +Join condition: None -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +(65) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] + +(66) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(67) CometColumnarExchange +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(68) CometColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(69) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] + +(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +(71) CometFilter +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) + +(72) CometColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] + +(73) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#85] + +(74) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] +Join type: Inner +Join condition: None + +(75) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] + +(76) ReusedExchange [Reuses operator id: 63] +Output [1]: [wp_web_page_sk#86] + +(77) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] +Join type: Inner +Join condition: None + +(78) Project [codegen id : 20] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] + +(79) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(80) CometColumnarExchange +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(81) CometColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(82) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] + +(83) BroadcastExchange +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(84) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] +Join type: LeftOuter +Join condition: None + +(85) Project [codegen id : 22] +Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] + +(86) Union + +(87) HashAggregate [codegen id : 23] +Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] +Keys [2]: [channel#31, id#32] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] (88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#116, returns#117, profit#118] -Keys: [] -Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(89) CometColumnarToRow [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(90) HashAggregate [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] + +(91) ReusedExchange [Reuses operator id: 88] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(92) CometColumnarToRow [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(93) HashAggregate [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] + +(94) HashAggregate [codegen id : 48] +Input [4]: [channel#31, sales#117, returns#118, profit#119] +Keys [1]: [channel#31] +Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] +Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] (95) CometColumnarExchange -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(96) CometColumnarToRow [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] + +(97) HashAggregate [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Keys [1]: [channel#31] +Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] +Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] + +(98) ReusedExchange [Reuses operator id: 88] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(99) CometColumnarToRow [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(100) HashAggregate [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [3]: [sum(sales#14)#111 AS sales#139, sum(returns#33)#112 AS returns#140, sum(profit#34)#113 AS profit#141] + +(101) HashAggregate [codegen id : 73] +Input [3]: [sales#139, returns#140, profit#141] +Keys: [] +Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] +Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Results [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] + +(102) CometColumnarExchange +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +(103) CometColumnarToRow [codegen id : 74] +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +(104) HashAggregate [codegen id : 74] +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Keys: [] -Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] -Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] -Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] +Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] +Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] +Results [5]: [null AS channel#157, null AS id#158, sum(sales#139)#154 AS sales#159, sum(returns#140)#155 AS returns#160, sum(profit#141)#156 AS profit#161] -(98) Union +(105) Union -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(106) HashAggregate [codegen id : 75] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(107) CometColumnarExchange +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(108) CometHashAggregate +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] +(109) CometTakeOrderedAndProject +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), [channel#31, id#32, sales#114, returns#115, profit#116], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(110) CometColumnarToRow [codegen id : 76] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) +BroadcastExchange (115) ++- * CometColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (111) -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] +(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#162] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(112) CometFilter +Input [2]: [d_date_sk#6, d_date#162] +Condition : (((isnotnull(d_date#162) AND (d_date#162 >= 1998-08-04)) AND (d_date#162 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] +(113) CometProject +Input [2]: [d_date_sk#6, d_date#162] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(114) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(108) BroadcastExchange +(115) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index 3b44c6c27c..9419932935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -9,114 +9,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -126,114 +144,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -243,113 +279,131 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 - +- CometColumnarToRow - +- 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 + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 143 out of 332 eligible operators (43%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt index ac3d312ee8..c67ddd5c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt @@ -1,139 +1,178 @@ -WholeStageCodegen (22) +WholeStageCodegen (76) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) + WholeStageCodegen (75) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (24) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) + WholeStageCodegen (23) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (14) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #14 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (49) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) + CometColumnarExchange [channel] #15 + WholeStageCodegen (48) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) + WholeStageCodegen (74) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) + CometColumnarExchange #16 + WholeStageCodegen (73) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt index 4b5efc84e3..5f6ee267fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt @@ -1,107 +1,114 @@ == Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) +* CometColumnarToRow (110) ++- CometTakeOrderedAndProject (109) + +- CometHashAggregate (108) + +- CometColumnarExchange (107) + +- * HashAggregate (106) + +- Union (105) :- * HashAggregate (90) : +- * CometColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) + : +- Union (86) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : : :- * HashAggregate (16) + : : : +- * CometColumnarToRow (15) + : : : +- CometColumnarExchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * CometColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * CometColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * CometColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : :- * Project (53) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) + : : :- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- * CometColumnarToRow (40) + : : : +- CometColumnarExchange (39) + : : : +- * HashAggregate (38) + : : : +- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * CometColumnarToRow (34) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) + : : : +- ReusedExchange (35) + : : +- * HashAggregate (51) + : : +- * CometColumnarToRow (50) + : : +- CometColumnarExchange (49) + : : +- * HashAggregate (48) + : : +- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * CometColumnarToRow (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : +- ReusedExchange (45) + : +- * Project (85) + : +- * BroadcastHashJoin LeftOuter BuildRight (84) + : :- * HashAggregate (69) + : : +- * CometColumnarToRow (68) + : : +- CometColumnarExchange (67) + : : +- * HashAggregate (66) + : : +- * Project (65) + : : +- * BroadcastHashJoin Inner BuildRight (64) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * CometColumnarToRow (56) + : : : : +- CometFilter (55) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (54) + : : : +- ReusedExchange (57) + : : +- BroadcastExchange (63) + : : +- * CometColumnarToRow (62) + : : +- CometFilter (61) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (60) + : +- BroadcastExchange (83) + : +- * HashAggregate (82) + : +- * CometColumnarToRow (81) + : +- CometColumnarExchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * CometColumnarToRow (72) + : : : +- CometFilter (71) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (70) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + :- * HashAggregate (97) + : +- * CometColumnarToRow (96) + : +- CometColumnarExchange (95) + : +- * HashAggregate (94) + : +- * HashAggregate (93) + : +- * CometColumnarToRow (92) + : +- ReusedExchange (91) + +- * HashAggregate (104) + +- * CometColumnarToRow (103) + +- CometColumnarExchange (102) + +- * HashAggregate (101) + +- * HashAggregate (100) + +- * CometColumnarToRow (99) + +- ReusedExchange (98) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,506 +123,556 @@ ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] +(3) CometColumnarToRow [codegen id : 3] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] +(4) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#6] -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None -(8) CometProject +(6) Project [codegen id : 3] +Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [1]: [s_store_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) +(8) CometFilter +Input [1]: [s_store_sk#7] +Condition : isnotnull(s_store_sk#7) + +(9) CometColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#7] -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] +(10) BroadcastExchange +Input [1]: [s_store_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +(12) Project [codegen id : 3] +Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] +(13) HashAggregate [codegen id : 3] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#8, sum#9] +Results [3]: [s_store_sk#7, sum#10, sum#11] -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(14) CometColumnarExchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] +(15) CometColumnarToRow [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] + +(16) HashAggregate [codegen id : 8] +Input [3]: [s_store_sk#7, sum#10, sum#11] +Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight +(19) CometColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +(20) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#21] -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] +(22) Project [codegen id : 6] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] +(23) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#22] -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] +(25) Project [codegen id : 6] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] + +(26) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] + +(27) CometColumnarExchange +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(28) CometColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] + +(29) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] + +(30) BroadcastExchange +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#22] +Join type: LeftOuter +Join condition: None -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +(32) Project [codegen id : 8] +Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] +(34) CometColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(35) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#40] -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] +(36) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +(37) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] + +(38) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(39) CometColumnarExchange +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(40) CometColumnarToRow [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(41) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] + +(42) BroadcastExchange +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] ReadSchema: struct -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] +(44) CometColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight +(45) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#53] -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] +(46) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] +(47) Project [codegen id : 13] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] + +(48) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#49, cr_net_loss#50] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(49) CometColumnarExchange +Input [2]: [sum#56, sum#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] +(50) CometColumnarToRow +Input [2]: [sum#56, sum#57] -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] +(51) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] -(49) BroadcastNestedLoopJoin [codegen id : 3] +(52) BroadcastNestedLoopJoin [codegen id : 14] Join type: Inner Join condition: None -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] +(53) Project [codegen id : 14] +Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) +(55) CometFilter +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) + +(56) CometColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] +(57) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#70] -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] +Join type: Inner +Join condition: None -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +(59) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#71] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) +(61) CometFilter +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] +(62) CometColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#71] -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight +(63) BroadcastExchange +Input [1]: [wp_web_page_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] +Join type: Inner +Join condition: None -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +(65) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] + +(66) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(67) CometColumnarExchange +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(68) CometColumnarToRow [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(69) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] + +(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +(71) CometFilter +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) + +(72) CometColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] + +(73) ReusedExchange [Reuses operator id: 115] +Output [1]: [d_date_sk#85] + +(74) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] +Join type: Inner +Join condition: None + +(75) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] + +(76) ReusedExchange [Reuses operator id: 63] +Output [1]: [wp_web_page_sk#86] + +(77) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] +Join type: Inner +Join condition: None + +(78) Project [codegen id : 20] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] + +(79) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(80) CometColumnarExchange +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(81) CometColumnarToRow [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(82) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] + +(83) BroadcastExchange +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(84) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] +Join type: LeftOuter +Join condition: None + +(85) Project [codegen id : 22] +Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] + +(86) Union + +(87) HashAggregate [codegen id : 23] +Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] +Keys [2]: [channel#31, id#32] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] (88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#116, returns#117, profit#118] -Keys: [] -Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(89) CometColumnarToRow [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(90) HashAggregate [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] + +(91) ReusedExchange [Reuses operator id: 88] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(92) CometColumnarToRow [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(93) HashAggregate [codegen id : 48] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] + +(94) HashAggregate [codegen id : 48] +Input [4]: [channel#31, sales#117, returns#118, profit#119] +Keys [1]: [channel#31] +Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] +Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] (95) CometColumnarExchange -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] + +(96) CometColumnarToRow [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] + +(97) HashAggregate [codegen id : 49] +Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] +Keys [1]: [channel#31] +Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] +Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] +Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] + +(98) ReusedExchange [Reuses operator id: 88] +Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(99) CometColumnarToRow [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(100) HashAggregate [codegen id : 73] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [3]: [sum(sales#14)#111 AS sales#139, sum(returns#33)#112 AS returns#140, sum(profit#34)#113 AS profit#141] + +(101) HashAggregate [codegen id : 73] +Input [3]: [sales#139, returns#140, profit#141] +Keys: [] +Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] +Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Results [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] + +(102) CometColumnarExchange +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +(103) CometColumnarToRow [codegen id : 74] +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +(104) HashAggregate [codegen id : 74] +Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] Keys: [] -Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] -Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] -Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] +Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] +Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] +Results [5]: [null AS channel#157, null AS id#158, sum(sales#139)#154 AS sales#159, sum(returns#140)#155 AS returns#160, sum(profit#141)#156 AS profit#161] -(98) Union +(105) Union -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(106) HashAggregate [codegen id : 75] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(107) CometColumnarExchange +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(108) CometHashAggregate +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] Functions: [] -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] +(109) CometTakeOrderedAndProject +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), [channel#31, id#32, sales#114, returns#115, profit#116], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +(110) CometColumnarToRow [codegen id : 76] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) +BroadcastExchange (115) ++- * CometColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (111) -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] +(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#162] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(112) CometFilter +Input [2]: [d_date_sk#6, d_date#162] +Condition : (((isnotnull(d_date#162) AND (d_date#162 >= 1998-08-04)) AND (d_date#162 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] +(113) CometProject +Input [2]: [d_date_sk#6, d_date#162] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(114) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(108) BroadcastExchange +(115) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt index 3b44c6c27c..9419932935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt @@ -9,114 +9,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -126,114 +144,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- 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 + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -243,113 +279,131 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 - : +- CometColumnarToRow - : +- 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 - +- CometColumnarToRow - +- 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 + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 143 out of 332 eligible operators (43%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index ac3d312ee8..c67ddd5c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -1,139 +1,178 @@ -WholeStageCodegen (22) +WholeStageCodegen (76) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) + WholeStageCodegen (75) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (24) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) + WholeStageCodegen (23) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,s_store_sk] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] - CometProject [sr_store_sk,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) + WholeStageCodegen (8) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (14) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] - CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #14 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (49) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) + CometColumnarExchange [channel] #15 + WholeStageCodegen (48) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) + WholeStageCodegen (74) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) + CometColumnarExchange #16 + WholeStageCodegen (73) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt index 82a5357f89..7578c94177 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt @@ -1,71 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- RowToColumnar (70) + +- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- * CometColumnarToRow (19) + : : +- CometColumnarExchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * CometColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * CometColumnarToRow (34) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (35) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- * CometColumnarToRow (64) + +- CometColumnarExchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * CometColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (48) + : : +- CometFilter (47) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (46) + : +- CometSort (54) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (50) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -124,282 +129,308 @@ Condition : isnull(sr_ticket_number#10) Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) +(13) CometColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] +(14) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#12, d_year#13] -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight +(15) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None -(17) CometProject +(16) Project [codegen id : 2] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) CometHashAggregate +(17) HashAggregate [codegen id : 2] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum#14, sum#15, sum#16] +Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(18) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(19) CometColumnarToRow [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +(20) HashAggregate [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] +(21) Sort [codegen id : 3] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) (24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] (25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) (28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] (29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] (30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] (31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter (32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) (33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(34) CometColumnarToRow [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(35) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#38, d_year#39] + +(36) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 5] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(38) HashAggregate [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(39) CometColumnarExchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(40) CometColumnarToRow [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(41) HashAggregate [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(42) Filter [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(43) Sort [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 7] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 7] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] + +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) +(47) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(48) CometExchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] +(49) CometSort +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) +(51) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) + +(52) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] + +(53) CometExchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(54) CometSort +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] +(55) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(56) CometFilter +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] +(57) CometProject +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter +(58) CometColumnarToRow [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) +(59) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#65, d_year#66] -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] +Join type: Inner +Join condition: None -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] +(61) Project [codegen id : 9] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight +(62) HashAggregate [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +(63) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] +(64) CometColumnarToRow [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(65) HashAggregate [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] +(66) Filter [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) +(67) Sort [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] +(68) SortMergeJoin [codegen id : 11] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner +(69) Project [codegen id : 11] +Output [13]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)))), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +(70) RowToColumnar +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +(71) CometTakeOrderedAndProject +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ss_sold_year#23 ASC NULLS FIRST,ss_item_sk#1 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST,ss_qty#24 DESC NULLS LAST,ss_wc#25 DESC NULLS LAST,ss_sp#26 DESC NULLS LAST,other_chan_qty#85 ASC NULLS FIRST,other_chan_wholesale_cost#86 ASC NULLS FIRST,other_chan_sales_price#87 ASC NULLS FIRST,ratio#81 ASC NULLS FIRST], output=[ratio#81,store_qty#82,store_wholesale_cost#83,store_sales_price#84,other_chan_qty#85,other_chan_wholesale_cost#86,other_chan_sales_price#87]), [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87], 100, 0, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] +(72) CometColumnarToRow [codegen id : 12] +Input [7]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(74) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index 97a1bdbbaf..f19cb299e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -1,80 +1,90 @@ - TakeOrderedAndProject [COMET: ] -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- SortMergeJoin + :- Project + : +- SortMergeJoin + : :- Sort + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometSortMergeJoin + : : : :- CometSort + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Sort + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- 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 + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Sort + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- 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 + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 48 out of 76 eligible operators (63%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt index 273db28e77..8f1fd0f9c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt @@ -1,78 +1,104 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) +WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ratio,store_qty,store_wholesale_cost,store_sales_price,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + RowToColumnar + WholeStageCodegen (11) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + InputAdapter + WholeStageCodegen (7) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (6) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (5) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (10) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (9) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometExchange [cs_order_number,cs_item_sk] #9 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_order_number,cr_item_sk] #10 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt index 82a5357f89..7578c94177 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt @@ -1,71 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- RowToColumnar (70) + +- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- * CometColumnarToRow (19) + : : +- CometColumnarExchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * CometColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : +- ReusedExchange (14) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * CometColumnarToRow (34) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (35) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- * CometColumnarToRow (64) + +- CometColumnarExchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * CometColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (48) + : : +- CometFilter (47) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (46) + : +- CometSort (54) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (50) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -124,282 +129,308 @@ Condition : isnull(sr_ticket_number#10) Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) +(13) CometColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] +(14) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#12, d_year#13] -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight +(15) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None -(17) CometProject +(16) Project [codegen id : 2] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(18) CometHashAggregate +(17) HashAggregate [codegen id : 2] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum#14, sum#15, sum#16] +Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(18) CometColumnarExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(19) CometColumnarToRow [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +(20) HashAggregate [codegen id : 3] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] +(21) Sort [codegen id : 3] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) (24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] (25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) (28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] (29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] (30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] (31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter +Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Right output [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter (32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) (33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(34) CometColumnarToRow [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(35) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#38, d_year#39] + +(36) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 5] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(38) HashAggregate [codegen id : 5] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(39) CometColumnarExchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(40) CometColumnarToRow [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(41) HashAggregate [codegen id : 6] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(42) Filter [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(43) Sort [codegen id : 6] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 7] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 7] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] + +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) +(47) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(48) CometExchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] +(49) CometSort +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) +(51) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) + +(52) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] + +(53) CometExchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(54) CometSort +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] +(55) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Right output [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(56) CometFilter +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] +(57) CometProject +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter +(58) CometColumnarToRow [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) +(59) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#65, d_year#66] -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] +Join type: Inner +Join condition: None -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] +(61) Project [codegen id : 9] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight +(62) HashAggregate [codegen id : 9] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +(63) CometColumnarExchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] +(64) CometColumnarToRow [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +(65) HashAggregate [codegen id : 10] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] +(66) Filter [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) +(67) Sort [codegen id : 10] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] +(68) SortMergeJoin [codegen id : 11] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner +(69) Project [codegen id : 11] +Output [13]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)))), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +(70) RowToColumnar +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +(71) CometTakeOrderedAndProject +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ss_sold_year#23 ASC NULLS FIRST,ss_item_sk#1 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST,ss_qty#24 DESC NULLS LAST,ss_wc#25 DESC NULLS LAST,ss_sp#26 DESC NULLS LAST,other_chan_qty#85 ASC NULLS FIRST,other_chan_wholesale_cost#86 ASC NULLS FIRST,other_chan_sales_price#87 ASC NULLS FIRST,ratio#81 ASC NULLS FIRST], output=[ratio#81,store_qty#82,store_wholesale_cost#83,store_sales_price#84,other_chan_qty#85,other_chan_wholesale_cost#86,other_chan_sales_price#87]), [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87], 100, 0, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] +(72) CometColumnarToRow [codegen id : 12] +Input [7]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter +(74) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(71) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt index 97a1bdbbaf..f19cb299e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt @@ -1,80 +1,90 @@ - TakeOrderedAndProject [COMET: ] -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- 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 - : : : : +- CometColumnarToRow - : : : : +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- SortMergeJoin + :- Project + : +- SortMergeJoin + : :- Sort + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometSortMergeJoin + : : : :- CometSort + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Sort + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- 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 + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Sort + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- 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 + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 48 out of 76 eligible operators (63%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 273db28e77..8f1fd0f9c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -1,78 +1,104 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) +WholeStageCodegen (12) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ratio,store_qty,store_wholesale_cost,store_sales_price,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + RowToColumnar + WholeStageCodegen (11) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + InputAdapter + WholeStageCodegen (7) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (3) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (2) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (6) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (5) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometExchange [ws_order_number,ws_item_sk] #6 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometExchange [wr_order_number,wr_item_sk] #7 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (10) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (9) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometExchange [cs_order_number,cs_item_sk] #9 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_order_number,cr_item_sk] #10 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt index f9901f13d7..e22170be2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt @@ -1,120 +1,133 @@ == Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) +* CometColumnarToRow (129) ++- CometTakeOrderedAndProject (128) + +- CometHashAggregate (127) + +- CometColumnarExchange (126) + +- * HashAggregate (125) + +- Union (124) + :- * HashAggregate (109) + : +- * CometColumnarToRow (108) + : +- CometColumnarExchange (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometColumnarExchange (38) + : : +- * HashAggregate (37) + : : +- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * CometColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- ReusedExchange (13) + : : : : +- BroadcastExchange (20) + : : : : +- * CometColumnarToRow (19) + : : : : +- CometProject (18) + : : : : +- CometFilter (17) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) + : : : +- BroadcastExchange (27) + : : : +- * CometColumnarToRow (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) + : : +- BroadcastExchange (34) + : : +- * CometColumnarToRow (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- * HashAggregate (72) + : : +- * CometColumnarToRow (71) + : : +- CometColumnarExchange (70) + : : +- * HashAggregate (69) + : : +- * Project (68) + : : +- * BroadcastHashJoin Inner BuildRight (67) + : : :- * Project (65) + : : : +- * BroadcastHashJoin Inner BuildRight (64) + : : : :- * Project (62) + : : : : +- * BroadcastHashJoin Inner BuildRight (61) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * CometColumnarToRow (52) + : : : : : : +- CometProject (51) + : : : : : : +- CometSortMergeJoin (50) + : : : : : : :- CometSort (44) + : : : : : : : +- CometExchange (43) + : : : : : : : +- CometFilter (42) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- CometSort (49) + : : : : : : +- CometExchange (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (60) + : : : : +- * CometColumnarToRow (59) + : : : : +- CometProject (58) + : : : : +- CometFilter (57) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (63) + : : +- ReusedExchange (66) + : +- * HashAggregate (104) + : +- * CometColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- * HashAggregate (101) + : +- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (87) + : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : :- * CometColumnarToRow (84) + : : : : : +- CometProject (83) + : : : : : +- CometSortMergeJoin (82) + : : : : : :- CometSort (76) + : : : : : : +- CometExchange (75) + : : : : : : +- CometFilter (74) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (73) + : : : : : +- CometSort (81) + : : : : : +- CometExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (77) + : : : : +- ReusedExchange (85) + : : : +- BroadcastExchange (92) + : : : +- * CometColumnarToRow (91) + : : : +- CometProject (90) + : : : +- CometFilter (89) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (88) + : : +- ReusedExchange (95) + : +- ReusedExchange (98) + :- * HashAggregate (116) + : +- * CometColumnarToRow (115) + : +- CometColumnarExchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- * CometColumnarToRow (111) + : +- ReusedExchange (110) + +- * HashAggregate (123) + +- * CometColumnarToRow (122) + +- CometColumnarExchange (121) + +- * HashAggregate (120) + +- * HashAggregate (119) + +- * CometColumnarToRow (118) + +- ReusedExchange (117) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -169,515 +182,583 @@ Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#1 Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(12) CometColumnarToRow [codegen id : 5] +Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(13) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#14] -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(17) CometProject +(15) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_store_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) +(17) CometFilter +Input [2]: [s_store_sk#15, s_store_id#16] +Condition : isnotnull(s_store_sk#15) -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] +(18) CometProject +Input [2]: [s_store_sk#15, s_store_id#16] +Arguments: [s_store_sk#15, s_store_id#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#17] -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] +(19) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#15, s_store_id#17] -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight +(20) BroadcastExchange +Input [2]: [s_store_sk#15, s_store_id#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] +(22) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#17] + +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) +(24) CometFilter +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) + +(25) CometProject +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] +(26) CometColumnarToRow [codegen id : 3] +Input [1]: [i_item_sk#18] -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] +(27) BroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(29) Project [codegen id : 5] +Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] (30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#21, 1, true, false, true) = N) AND isnotnull(p_promo_sk#20)) (32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] +(33) CometColumnarToRow [codegen id : 4] +Input [1]: [p_promo_sk#20] -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight +(34) BroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(35) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_promo_sk#3] +Right keys [1]: [p_promo_sk#20] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] +(36) Project [codegen id : 5] +Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] + +(37) HashAggregate [codegen id : 5] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(38) CometColumnarExchange +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(39) CometColumnarToRow [codegen id : 6] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] +(40) HashAggregate [codegen id : 6] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] +Results [5]: [store channel AS channel#35, concat(store, s_store_id#17) AS id#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#37, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#38, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#39] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) +(42) CometFilter +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(43) CometExchange +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] +(44) CometSort +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) +(46) CometFilter +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) + +(47) CometProject +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +(48) CometExchange +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(49) CometSort +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] +(50) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter +(51) CometProject +Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +(52) CometColumnarToRow [codegen id : 11] +Input [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] +(53) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#53] -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight +(54) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +(55) Project [codegen id : 11] +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight +(57) CometFilter +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Condition : isnotnull(cp_catalog_page_sk#54) (58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#56], [cp_catalog_page_sk#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#55, 16, true, false, true) AS cp_catalog_page_id#56] + +(59) CometColumnarToRow [codegen id : 8] +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] + +(60) BroadcastExchange +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(61) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_catalog_page_sk#40] +Right keys [1]: [cp_catalog_page_sk#54] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 11] +Output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#56] + +(63) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#57] + +(64) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#41] +Right keys [1]: [i_item_sk#57] +Join type: Inner +Join condition: None + +(65) Project [codegen id : 11] +Output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, i_item_sk#57] + +(66) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#58] + +(67) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_promo_sk#42] +Right keys [1]: [p_promo_sk#58] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 11] +Output [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, p_promo_sk#58] + +(69) HashAggregate [codegen id : 11] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Keys [1]: [cp_catalog_page_id#56] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#59, sum#60, isEmpty#61, sum#62, isEmpty#63] +Results [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] + +(70) CometColumnarExchange +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cp_catalog_page_id#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(71) CometColumnarToRow [codegen id : 12] +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] + +(72) HashAggregate [codegen id : 12] +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Keys [1]: [cp_catalog_page_id#56] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#69, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71] +Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#56) AS id#73, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#69,17,2) AS sales#74, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70 AS returns#75, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71 AS profit#76] + +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) +(74) CometFilter +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Condition : ((isnotnull(ws_web_site_sk#78) AND isnotnull(ws_item_sk#77)) AND isnotnull(ws_promo_sk#79)) -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(75) CometExchange +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Arguments: hashpartitioning(ws_item_sk#77, ws_order_number#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] +(76) CometSort +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83], [ws_item_sk#77 ASC NULLS FIRST, ws_order_number#80 ASC NULLS FIRST] -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) +(78) CometFilter +Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +Condition : (isnotnull(wr_item_sk#85) AND isnotnull(wr_order_number#86)) -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +(79) CometProject +Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(80) CometExchange +Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: hashpartitioning(wr_item_sk#85, wr_order_number#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] +(81) CometSort +Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85 ASC NULLS FIRST, wr_order_number#86 ASC NULLS FIRST] + +(82) CometSortMergeJoin +Left output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Right output [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [ws_item_sk#77, ws_order_number#80], [wr_item_sk#85, wr_order_number#86], LeftOuter -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter +(83) CometProject +Input [11]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88], [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +(84) CometColumnarToRow [codegen id : 17] +Input [8]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] +(85) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#90] -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight +(86) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] +Join type: Inner +Join condition: None -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +(87) Project [codegen id : 17] +Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88] +Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88, d_date_sk#90] -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] +(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#91, web_site_id#92] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight +(89) CometFilter +Input [2]: [web_site_sk#91, web_site_id#92] +Condition : isnotnull(web_site_sk#91) (90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#106, returns#107, profit#108] +Input [2]: [web_site_sk#91, web_site_id#92] +Arguments: [web_site_sk#91, web_site_id#93], [web_site_sk#91, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#92, 16, true, false, true) AS web_site_id#93] + +(91) CometColumnarToRow [codegen id : 14] +Input [2]: [web_site_sk#91, web_site_id#93] + +(92) BroadcastExchange +Input [2]: [web_site_sk#91, web_site_id#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +(93) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_site_sk#78] +Right keys [1]: [web_site_sk#91] +Join type: Inner +Join condition: None + +(94) Project [codegen id : 17] +Output [7]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_sk#91, web_site_id#93] + +(95) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#94] + +(96) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#77] +Right keys [1]: [i_item_sk#94] +Join type: Inner +Join condition: None + +(97) Project [codegen id : 17] +Output [6]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [8]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, i_item_sk#94] + +(98) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#95] + +(99) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_promo_sk#79] +Right keys [1]: [p_promo_sk#95] +Join type: Inner +Join condition: None + +(100) Project [codegen id : 17] +Output [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [7]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, p_promo_sk#95] + +(101) HashAggregate [codegen id : 17] +Input [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Keys [1]: [web_site_id#93] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#81)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] + +(102) CometColumnarExchange +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(web_site_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(103) CometColumnarToRow [codegen id : 18] +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] + +(104) HashAggregate [codegen id : 18] +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [web_site_id#93] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#81)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#81))#106, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108] +Results [5]: [web channel AS channel#109, concat(web_site, web_site_id#93) AS id#110, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#106,17,2) AS sales#111, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107 AS returns#112, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108 AS profit#113] + +(105) Union + +(106) HashAggregate [codegen id : 19] +Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] +Keys [2]: [channel#35, id#36] +Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Results [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(107) CometColumnarExchange +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(108) CometColumnarToRow [codegen id : 20] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(109) HashAggregate [codegen id : 20] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [5]: [channel#35, id#36, cast(sum(sales#37)#126 as decimal(37,2)) AS sales#129, cast(sum(returns#38)#127 as decimal(38,2)) AS returns#130, cast(sum(profit#39)#128 as decimal(38,2)) AS profit#131] + +(110) ReusedExchange [Reuses operator id: 107] +Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(111) CometColumnarToRow [codegen id : 40] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(112) HashAggregate [codegen id : 40] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [4]: [channel#35, sum(sales#37)#126 AS sales#132, sum(returns#38)#127 AS returns#133, sum(profit#39)#128 AS profit#134] + +(113) HashAggregate [codegen id : 40] +Input [4]: [channel#35, sales#132, returns#133, profit#134] +Keys [1]: [channel#35] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] + +(114) CometColumnarExchange +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(115) CometColumnarToRow [codegen id : 41] +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] + +(116) HashAggregate [codegen id : 41] +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#35] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#147, sum(returns#133)#148, sum(profit#134)#149] +Results [5]: [channel#35, null AS id#150, sum(sales#132)#147 AS sales#151, sum(returns#133)#148 AS returns#152, sum(profit#134)#149 AS profit#153] + +(117) ReusedExchange [Reuses operator id: 107] +Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(118) CometColumnarToRow [codegen id : 61] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(119) HashAggregate [codegen id : 61] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [3]: [sum(sales#37)#126 AS sales#154, sum(returns#38)#127 AS returns#155, sum(profit#39)#128 AS profit#156] + +(120) HashAggregate [codegen id : 61] +Input [3]: [sales#154, returns#155, profit#156] Keys: [] -Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] +Aggregate Attributes [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Results [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] + +(121) CometColumnarExchange +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(109) CometExchange -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(122) CometColumnarToRow [codegen id : 62] +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] -(110) CometHashAggregate -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +(123) HashAggregate [codegen id : 62] +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] Keys: [] -Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] +Aggregate Attributes [3]: [sum(sales#154)#169, sum(returns#155)#170, sum(profit#156)#171] +Results [5]: [null AS channel#172, null AS id#173, sum(sales#154)#169 AS sales#174, sum(returns#155)#170 AS returns#175, sum(profit#156)#171 AS profit#176] -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] -Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] +(124) Union -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(125) HashAggregate [codegen id : 63] +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -(113) CometExchange -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(126) CometColumnarExchange +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Arguments: hashpartitioning(channel#35, id#36, sales#129, returns#130, profit#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(127) CometHashAggregate +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] Functions: [] -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] +(128) CometTakeOrderedAndProject +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#129,returns#130,profit#131]), [channel#35, id#36, sales#129, returns#130, profit#131], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#129, returns#130, profit#131] -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(129) CometColumnarToRow [codegen id : 64] +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) +BroadcastExchange (134) ++- * CometColumnarToRow (133) + +- CometProject (132) + +- CometFilter (131) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (130) -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(130) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#177] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(131) CometFilter +Input [2]: [d_date_sk#14, d_date#177] +Condition : (((isnotnull(d_date#177) AND (d_date#177 >= 1998-08-04)) AND (d_date#177 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(132) CometProject +Input [2]: [d_date_sk#14, d_date#177] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(133) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(121) BroadcastExchange +(134) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..7c1d834ff9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -1,392 +1,451 @@ CometColumnarToRow +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometSortMergeJoin + : : : : : : :- CometSort + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometSortMergeJoin + : : : : : : :- CometSort + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 233 out of 386 eligible operators (60%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt index d27ab827e1..18b1a637b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt @@ -1,128 +1,189 @@ -WholeStageCodegen (1) +WholeStageCodegen (64) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (63) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (20) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (19) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (5) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (12) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + WholeStageCodegen (11) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (18) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + WholeStageCodegen (17) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (14) + CometColumnarToRow + InputAdapter + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (41) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #18 + WholeStageCodegen (40) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (62) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #19 + WholeStageCodegen (61) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index f9901f13d7..e22170be2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -1,120 +1,133 @@ == Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) +* CometColumnarToRow (129) ++- CometTakeOrderedAndProject (128) + +- CometHashAggregate (127) + +- CometColumnarExchange (126) + +- * HashAggregate (125) + +- Union (124) + :- * HashAggregate (109) + : +- * CometColumnarToRow (108) + : +- CometColumnarExchange (107) + : +- * HashAggregate (106) + : +- Union (105) + : :- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometColumnarExchange (38) + : : +- * HashAggregate (37) + : : +- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * CometColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- ReusedExchange (13) + : : : : +- BroadcastExchange (20) + : : : : +- * CometColumnarToRow (19) + : : : : +- CometProject (18) + : : : : +- CometFilter (17) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) + : : : +- BroadcastExchange (27) + : : : +- * CometColumnarToRow (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) + : : +- BroadcastExchange (34) + : : +- * CometColumnarToRow (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- * HashAggregate (72) + : : +- * CometColumnarToRow (71) + : : +- CometColumnarExchange (70) + : : +- * HashAggregate (69) + : : +- * Project (68) + : : +- * BroadcastHashJoin Inner BuildRight (67) + : : :- * Project (65) + : : : +- * BroadcastHashJoin Inner BuildRight (64) + : : : :- * Project (62) + : : : : +- * BroadcastHashJoin Inner BuildRight (61) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * CometColumnarToRow (52) + : : : : : : +- CometProject (51) + : : : : : : +- CometSortMergeJoin (50) + : : : : : : :- CometSort (44) + : : : : : : : +- CometExchange (43) + : : : : : : : +- CometFilter (42) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- CometSort (49) + : : : : : : +- CometExchange (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (60) + : : : : +- * CometColumnarToRow (59) + : : : : +- CometProject (58) + : : : : +- CometFilter (57) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (63) + : : +- ReusedExchange (66) + : +- * HashAggregate (104) + : +- * CometColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- * HashAggregate (101) + : +- * Project (100) + : +- * BroadcastHashJoin Inner BuildRight (99) + : :- * Project (97) + : : +- * BroadcastHashJoin Inner BuildRight (96) + : : :- * Project (94) + : : : +- * BroadcastHashJoin Inner BuildRight (93) + : : : :- * Project (87) + : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : :- * CometColumnarToRow (84) + : : : : : +- CometProject (83) + : : : : : +- CometSortMergeJoin (82) + : : : : : :- CometSort (76) + : : : : : : +- CometExchange (75) + : : : : : : +- CometFilter (74) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (73) + : : : : : +- CometSort (81) + : : : : : +- CometExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (77) + : : : : +- ReusedExchange (85) + : : : +- BroadcastExchange (92) + : : : +- * CometColumnarToRow (91) + : : : +- CometProject (90) + : : : +- CometFilter (89) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (88) + : : +- ReusedExchange (95) + : +- ReusedExchange (98) + :- * HashAggregate (116) + : +- * CometColumnarToRow (115) + : +- CometColumnarExchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- * CometColumnarToRow (111) + : +- ReusedExchange (110) + +- * HashAggregate (123) + +- * CometColumnarToRow (122) + +- CometColumnarExchange (121) + +- * HashAggregate (120) + +- * HashAggregate (119) + +- * CometColumnarToRow (118) + +- ReusedExchange (117) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -169,515 +182,583 @@ Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#1 Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(12) CometColumnarToRow [codegen id : 5] +Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(13) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#14] -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(17) CometProject +(15) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_store_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) +(17) CometFilter +Input [2]: [s_store_sk#15, s_store_id#16] +Condition : isnotnull(s_store_sk#15) -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] +(18) CometProject +Input [2]: [s_store_sk#15, s_store_id#16] +Arguments: [s_store_sk#15, s_store_id#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#17] -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] +(19) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#15, s_store_id#17] -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight +(20) BroadcastExchange +Input [2]: [s_store_sk#15, s_store_id#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] +(22) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#17] + +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#18, i_current_price#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) +(24) CometFilter +Input [2]: [i_item_sk#18, i_current_price#19] +Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) + +(25) CometProject +Input [2]: [i_item_sk#18, i_current_price#19] +Arguments: [i_item_sk#18], [i_item_sk#18] -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] +(26) CometColumnarToRow [codegen id : 3] +Input [1]: [i_item_sk#18] -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] +(27) BroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight +(28) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(29) Project [codegen id : 5] +Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] (30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] +Output [2]: [p_promo_sk#20, p_channel_tv#21] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#21, 1, true, false, true) = N) AND isnotnull(p_promo_sk#20)) (32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] +Input [2]: [p_promo_sk#20, p_channel_tv#21] +Arguments: [p_promo_sk#20], [p_promo_sk#20] -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] +(33) CometColumnarToRow [codegen id : 4] +Input [1]: [p_promo_sk#20] -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight +(34) BroadcastExchange +Input [1]: [p_promo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +(35) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_promo_sk#3] +Right keys [1]: [p_promo_sk#20] +Join type: Inner +Join condition: None -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] +(36) Project [codegen id : 5] +Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] + +(37) HashAggregate [codegen id : 5] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] +Keys [1]: [s_store_id#17] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] +Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(38) CometColumnarExchange +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(39) CometColumnarToRow [codegen id : 6] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] +(40) HashAggregate [codegen id : 6] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +Keys [1]: [s_store_id#17] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] +Results [5]: [store channel AS channel#35, concat(store, s_store_id#17) AS id#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#37, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#38, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#39] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) +(42) CometFilter +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(43) CometExchange +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] +(44) CometSort +Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) +(46) CometFilter +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) + +(47) CometProject +Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +(48) CometExchange +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(49) CometSort +Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] +(50) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter +(51) CometProject +Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +(52) CometColumnarToRow [codegen id : 11] +Input [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] +(53) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#53] -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight +(54) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +(55) Project [codegen id : 11] +Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight +(57) CometFilter +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Condition : isnotnull(cp_catalog_page_sk#54) (58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#56], [cp_catalog_page_sk#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#55, 16, true, false, true) AS cp_catalog_page_id#56] + +(59) CometColumnarToRow [codegen id : 8] +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] + +(60) BroadcastExchange +Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(61) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_catalog_page_sk#40] +Right keys [1]: [cp_catalog_page_sk#54] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 11] +Output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#56] + +(63) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#57] + +(64) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#41] +Right keys [1]: [i_item_sk#57] +Join type: Inner +Join condition: None + +(65) Project [codegen id : 11] +Output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, i_item_sk#57] + +(66) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#58] + +(67) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_promo_sk#42] +Right keys [1]: [p_promo_sk#58] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 11] +Output [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, p_promo_sk#58] + +(69) HashAggregate [codegen id : 11] +Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] +Keys [1]: [cp_catalog_page_id#56] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#59, sum#60, isEmpty#61, sum#62, isEmpty#63] +Results [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] + +(70) CometColumnarExchange +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cp_catalog_page_id#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(71) CometColumnarToRow [codegen id : 12] +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] + +(72) HashAggregate [codegen id : 12] +Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Keys [1]: [cp_catalog_page_id#56] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#69, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71] +Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#56) AS id#73, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#69,17,2) AS sales#74, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70 AS returns#75, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71 AS profit#76] + +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) +(74) CometFilter +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Condition : ((isnotnull(ws_web_site_sk#78) AND isnotnull(ws_item_sk#77)) AND isnotnull(ws_promo_sk#79)) -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(75) CometExchange +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Arguments: hashpartitioning(ws_item_sk#77, ws_order_number#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] +(76) CometSort +Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83], [ws_item_sk#77 ASC NULLS FIRST, ws_order_number#80 ASC NULLS FIRST] -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) +(78) CometFilter +Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +Condition : (isnotnull(wr_item_sk#85) AND isnotnull(wr_order_number#86)) -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +(79) CometProject +Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(80) CometExchange +Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: hashpartitioning(wr_item_sk#85, wr_order_number#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] +(81) CometSort +Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85 ASC NULLS FIRST, wr_order_number#86 ASC NULLS FIRST] + +(82) CometSortMergeJoin +Left output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Right output [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [ws_item_sk#77, ws_order_number#80], [wr_item_sk#85, wr_order_number#86], LeftOuter -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter +(83) CometProject +Input [11]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88], [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +(84) CometColumnarToRow [codegen id : 17] +Input [8]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] +(85) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#90] -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight +(86) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] +Join type: Inner +Join condition: None -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +(87) Project [codegen id : 17] +Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88] +Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88, d_date_sk#90] -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] +(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#91, web_site_id#92] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight +(89) CometFilter +Input [2]: [web_site_sk#91, web_site_id#92] +Condition : isnotnull(web_site_sk#91) (90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#106, returns#107, profit#108] +Input [2]: [web_site_sk#91, web_site_id#92] +Arguments: [web_site_sk#91, web_site_id#93], [web_site_sk#91, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#92, 16, true, false, true) AS web_site_id#93] + +(91) CometColumnarToRow [codegen id : 14] +Input [2]: [web_site_sk#91, web_site_id#93] + +(92) BroadcastExchange +Input [2]: [web_site_sk#91, web_site_id#93] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +(93) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_site_sk#78] +Right keys [1]: [web_site_sk#91] +Join type: Inner +Join condition: None + +(94) Project [codegen id : 17] +Output [7]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_sk#91, web_site_id#93] + +(95) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#94] + +(96) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#77] +Right keys [1]: [i_item_sk#94] +Join type: Inner +Join condition: None + +(97) Project [codegen id : 17] +Output [6]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [8]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, i_item_sk#94] + +(98) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#95] + +(99) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_promo_sk#79] +Right keys [1]: [p_promo_sk#95] +Join type: Inner +Join condition: None + +(100) Project [codegen id : 17] +Output [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Input [7]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, p_promo_sk#95] + +(101) HashAggregate [codegen id : 17] +Input [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] +Keys [1]: [web_site_id#93] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#81)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] + +(102) CometColumnarExchange +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(web_site_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] + +(103) CometColumnarToRow [codegen id : 18] +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] + +(104) HashAggregate [codegen id : 18] +Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [web_site_id#93] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#81)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#81))#106, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108] +Results [5]: [web channel AS channel#109, concat(web_site, web_site_id#93) AS id#110, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#106,17,2) AS sales#111, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107 AS returns#112, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108 AS profit#113] + +(105) Union + +(106) HashAggregate [codegen id : 19] +Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] +Keys [2]: [channel#35, id#36] +Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Results [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(107) CometColumnarExchange +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] + +(108) CometColumnarToRow [codegen id : 20] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(109) HashAggregate [codegen id : 20] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [5]: [channel#35, id#36, cast(sum(sales#37)#126 as decimal(37,2)) AS sales#129, cast(sum(returns#38)#127 as decimal(38,2)) AS returns#130, cast(sum(profit#39)#128 as decimal(38,2)) AS profit#131] + +(110) ReusedExchange [Reuses operator id: 107] +Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(111) CometColumnarToRow [codegen id : 40] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(112) HashAggregate [codegen id : 40] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [4]: [channel#35, sum(sales#37)#126 AS sales#132, sum(returns#38)#127 AS returns#133, sum(profit#39)#128 AS profit#134] + +(113) HashAggregate [codegen id : 40] +Input [4]: [channel#35, sales#132, returns#133, profit#134] +Keys [1]: [channel#35] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] + +(114) CometColumnarExchange +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(115) CometColumnarToRow [codegen id : 41] +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] + +(116) HashAggregate [codegen id : 41] +Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#35] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#147, sum(returns#133)#148, sum(profit#134)#149] +Results [5]: [channel#35, null AS id#150, sum(sales#132)#147 AS sales#151, sum(returns#133)#148 AS returns#152, sum(profit#134)#149 AS profit#153] + +(117) ReusedExchange [Reuses operator id: 107] +Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(118) CometColumnarToRow [codegen id : 61] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(119) HashAggregate [codegen id : 61] +Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#35, id#36] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] +Results [3]: [sum(sales#37)#126 AS sales#154, sum(returns#38)#127 AS returns#155, sum(profit#39)#128 AS profit#156] + +(120) HashAggregate [codegen id : 61] +Input [3]: [sales#154, returns#155, profit#156] Keys: [] -Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] +Aggregate Attributes [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Results [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] + +(121) CometColumnarExchange +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] -(109) CometExchange -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(122) CometColumnarToRow [codegen id : 62] +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] -(110) CometHashAggregate -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +(123) HashAggregate [codegen id : 62] +Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] Keys: [] -Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] +Aggregate Attributes [3]: [sum(sales#154)#169, sum(returns#155)#170, sum(profit#156)#171] +Results [5]: [null AS channel#172, null AS id#173, sum(sales#154)#169 AS sales#174, sum(returns#155)#170 AS returns#175, sum(profit#156)#171 AS profit#176] -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] -Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] +(124) Union -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(125) HashAggregate [codegen id : 63] +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -(113) CometExchange -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(126) CometColumnarExchange +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Arguments: hashpartitioning(channel#35, id#36, sales#129, returns#130, profit#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(127) CometHashAggregate +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] Functions: [] -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] +(128) CometTakeOrderedAndProject +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#129,returns#130,profit#131]), [channel#35, id#36, sales#129, returns#130, profit#131], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#129, returns#130, profit#131] -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +(129) CometColumnarToRow [codegen id : 64] +Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) +BroadcastExchange (134) ++- * CometColumnarToRow (133) + +- CometProject (132) + +- CometFilter (131) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (130) -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +(130) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#177] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(131) CometFilter +Input [2]: [d_date_sk#14, d_date#177] +Condition : (((isnotnull(d_date#177) AND (d_date#177 >= 1998-08-04)) AND (d_date#177 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] +(132) CometProject +Input [2]: [d_date_sk#14, d_date#177] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(133) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(121) BroadcastExchange +(134) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt index 0d6844d93c..7c1d834ff9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt @@ -1,392 +1,451 @@ CometColumnarToRow +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometSortMergeJoin + : : : : : : :- CometSort + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow : +- 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 - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometSortMergeJoin + : : : : : : :- CometSort + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- 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 + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 233 out of 386 eligible operators (60%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index d27ab827e1..18b1a637b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -1,128 +1,189 @@ -WholeStageCodegen (1) +WholeStageCodegen (64) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (63) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (20) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (19) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (5) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (12) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + WholeStageCodegen (11) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (18) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + WholeStageCodegen (17) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (14) + CometColumnarToRow + InputAdapter + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (41) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #18 + WholeStageCodegen (40) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (62) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #19 + WholeStageCodegen (61) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt index b623de85d0..586f9cd0f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt @@ -1,41 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) +* CometColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- RowToColumnar (41) + +- * Project (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- Union (32) + :- * HashAggregate (17) + : +- * CometColumnarToRow (16) + : +- CometColumnarExchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (11) + : +- * CometColumnarToRow (10) + : +- CometProject (9) + : +- CometFilter (8) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (19) + : +- ReusedExchange (18) + +- * HashAggregate (31) + +- * CometColumnarToRow (30) + +- CometColumnarExchange (29) + +- * HashAggregate (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- ReusedExchange (25) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -50,192 +56,220 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject +(6) Project [codegen id : 3] +Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#6, i_class#7, i_category#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] +(8) CometFilter +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Condition : isnotnull(i_item_sk#6) -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] +(9) CometProject +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight +(10) CometColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#6, i_class#9, i_category#10] -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] +(11) BroadcastExchange +Input [3]: [i_item_sk#6, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +(13) Project [codegen id : 3] +Output [3]: [ws_net_paid#2, i_class#9, i_category#10] +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] +(14) HashAggregate [codegen id : 3] +Input [3]: [ws_net_paid#2, i_class#9, i_category#10] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum#11] +Results [3]: [i_category#10, i_class#9, sum#12] -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] +(15) CometColumnarExchange +Input [3]: [i_category#10, i_class#9, sum#12] +Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(16) CometColumnarToRow [codegen id : 4] +Input [3]: [i_category#10, i_class#9, sum#12] -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] +(17) HashAggregate [codegen id : 4] +Input [3]: [i_category#10, i_class#9, sum#12] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] + +(18) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#10, i_class#9, sum#20] + +(19) CometColumnarToRow [codegen id : 8] +Input [3]: [i_category#10, i_class#9, sum#20] + +(20) HashAggregate [codegen id : 8] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] + +(21) HashAggregate [codegen id : 8] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] + +(22) CometColumnarExchange +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometColumnarToRow [codegen id : 9] +Input [3]: [i_category#10, sum#25, isEmpty#26] + +(24) HashAggregate [codegen id : 9] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#27] +Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] + +(25) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#10, i_class#9, sum#33] + +(26) CometColumnarToRow [codegen id : 13] +Input [3]: [i_category#10, i_class#9, sum#33] + +(27) HashAggregate [codegen id : 13] +Input [3]: [i_category#10, i_class#9, sum#33] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] + +(28) HashAggregate [codegen id : 13] +Input [1]: [total_sum#35] +Keys: [] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] +(29) CometColumnarExchange +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] +(30) CometColumnarToRow [codegen id : 14] +Input [2]: [sum#38, isEmpty#39] -(25) CometHashAggregate -Input [1]: [total_sum#20] +(31) HashAggregate [codegen id : 14] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [partial_sum(total_sum#20)] +Functions [1]: [sum(total_sum#35)] +Aggregate Attributes [1]: [sum(total_sum#35)#40] +Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] -(26) CometExchange -Input [2]: [sum#21, isEmpty#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(32) Union -(27) CometHashAggregate -Input [2]: [sum#21, isEmpty#22] -Keys: [] -Functions [1]: [sum(total_sum#20)] +(33) HashAggregate [codegen id : 15] +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -(28) CometUnion -Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] -Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] +(34) CometColumnarExchange +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +(35) CometHashAggregate +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Functions: [] -(30) CometExchange -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(36) CometExchange +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(31) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] +(37) CometSort +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(32) CometExchange -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(38) CometColumnarToRow [codegen id : 16] +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -(33) CometSort -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] +(39) Window +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +(40) Project [codegen id : 17] +Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] -(35) Window -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] +(41) RowToColumnar +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -(36) Project [codegen id : 2] -Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] +(42) CometTakeOrderedAndProject +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#48 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#48]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -(37) TakeOrderedAndProject -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +(43) CometColumnarToRow [codegen id : 18] +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(45) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#49] +Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +(46) CometProject +Input [2]: [d_date_sk#5, d_month_seq#49] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(42) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 6dd65034ce..156bf16204 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -1,87 +1,103 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometExchange +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 81 eligible operators (53%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt index 11e3c03d80..c53fb1296c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt @@ -1,49 +1,75 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (18) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (17) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (15) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (9) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #6 + WholeStageCodegen (8) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (14) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #7 + WholeStageCodegen (13) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt index b623de85d0..586f9cd0f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt @@ -1,41 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) +* CometColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- RowToColumnar (41) + +- * Project (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- Union (32) + :- * HashAggregate (17) + : +- * CometColumnarToRow (16) + : +- CometColumnarExchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * CometColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (11) + : +- * CometColumnarToRow (10) + : +- CometProject (9) + : +- CometFilter (8) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (19) + : +- ReusedExchange (18) + +- * HashAggregate (31) + +- * CometColumnarToRow (30) + +- CometColumnarExchange (29) + +- * HashAggregate (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- ReusedExchange (25) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -50,192 +56,220 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] +(4) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#5] -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject +(6) Project [codegen id : 3] +Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#6, i_class#7, i_category#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] +(8) CometFilter +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Condition : isnotnull(i_item_sk#6) -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] +(9) CometProject +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight +(10) CometColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#6, i_class#9, i_category#10] -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] +(11) BroadcastExchange +Input [3]: [i_item_sk#6, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +(13) Project [codegen id : 3] +Output [3]: [ws_net_paid#2, i_class#9, i_category#10] +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] +(14) HashAggregate [codegen id : 3] +Input [3]: [ws_net_paid#2, i_class#9, i_category#10] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum#11] +Results [3]: [i_category#10, i_class#9, sum#12] -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] +(15) CometColumnarExchange +Input [3]: [i_category#10, i_class#9, sum#12] +Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(16) CometColumnarToRow [codegen id : 4] +Input [3]: [i_category#10, i_class#9, sum#12] -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] +(17) HashAggregate [codegen id : 4] +Input [3]: [i_category#10, i_class#9, sum#12] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] + +(18) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#10, i_class#9, sum#20] + +(19) CometColumnarToRow [codegen id : 8] +Input [3]: [i_category#10, i_class#9, sum#20] + +(20) HashAggregate [codegen id : 8] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] + +(21) HashAggregate [codegen id : 8] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] + +(22) CometColumnarExchange +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(23) CometColumnarToRow [codegen id : 9] +Input [3]: [i_category#10, sum#25, isEmpty#26] + +(24) HashAggregate [codegen id : 9] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#27] +Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] + +(25) ReusedExchange [Reuses operator id: 15] +Output [3]: [i_category#10, i_class#9, sum#33] + +(26) CometColumnarToRow [codegen id : 13] +Input [3]: [i_category#10, i_class#9, sum#33] + +(27) HashAggregate [codegen id : 13] +Input [3]: [i_category#10, i_class#9, sum#33] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] + +(28) HashAggregate [codegen id : 13] +Input [1]: [total_sum#35] +Keys: [] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] +(29) CometColumnarExchange +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] +(30) CometColumnarToRow [codegen id : 14] +Input [2]: [sum#38, isEmpty#39] -(25) CometHashAggregate -Input [1]: [total_sum#20] +(31) HashAggregate [codegen id : 14] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [partial_sum(total_sum#20)] +Functions [1]: [sum(total_sum#35)] +Aggregate Attributes [1]: [sum(total_sum#35)#40] +Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] -(26) CometExchange -Input [2]: [sum#21, isEmpty#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(32) Union -(27) CometHashAggregate -Input [2]: [sum#21, isEmpty#22] -Keys: [] -Functions [1]: [sum(total_sum#20)] +(33) HashAggregate [codegen id : 15] +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -(28) CometUnion -Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] -Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] +(34) CometColumnarExchange +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +(35) CometHashAggregate +Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Functions: [] -(30) CometExchange -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(36) CometExchange +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(31) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] +(37) CometSort +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(32) CometExchange -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(38) CometColumnarToRow [codegen id : 16] +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -(33) CometSort -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] +(39) Window +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +(40) Project [codegen id : 17] +Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] -(35) Window -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] +(41) RowToColumnar +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -(36) Project [codegen id : 2] -Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] +(42) CometTakeOrderedAndProject +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#48 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#48]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -(37) TakeOrderedAndProject -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +(43) CometColumnarToRow [codegen id : 18] +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#49] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) +(45) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#49] +Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] +(46) CometProject +Input [2]: [d_date_sk#5, d_month_seq#49] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(42) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt index 6dd65034ce..156bf16204 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt @@ -1,87 +1,103 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate +CometColumnarToRow ++- CometTakeOrderedAndProject + +- RowToColumnar + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometExchange +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometColumnarToRow - : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 43 out of 81 eligible operators (53%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index 11e3c03d80..c53fb1296c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -1,49 +1,75 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow +WholeStageCodegen (18) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + RowToColumnar + WholeStageCodegen (17) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (16) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (15) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (9) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #6 + WholeStageCodegen (8) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (14) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #7 + WholeStageCodegen (13) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt index 96a0ec392e..38871726a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt @@ -6,24 +6,24 @@ +- Window (21) +- * CometColumnarToRow (20) +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -38,105 +38,104 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(15) CometColumnarExchange +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometExchange +(16) CometColumnarToRow [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometHashAggregate +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] (23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== @@ -149,18 +148,18 @@ BroadcastExchange (30) (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct (27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) (28) CometProject -Input [2]: [d_date_sk#14, d_date#15] +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] (29) CometColumnarToRow [codegen id : 1] @@ -168,6 +167,6 @@ Input [1]: [d_date_sk#14] (30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 54a4d3c1b7..58cabe9418 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -5,29 +5,33 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 16 out of 28 eligible operators (57%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt index 46191f59cd..788b3f8f4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt @@ -1,39 +1,47 @@ -WholeStageCodegen (3) +WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) + WholeStageCodegen (6) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (1) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt index 96a0ec392e..38871726a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt @@ -6,24 +6,24 @@ +- Window (21) +- * CometColumnarToRow (20) +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometColumnarExchange (18) + +- * HashAggregate (17) + +- * CometColumnarToRow (16) + +- CometColumnarExchange (15) + +- * HashAggregate (14) + +- * Project (13) + +- * BroadcastHashJoin Inner BuildRight (12) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * CometColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (8) + : +- * CometColumnarToRow (7) + : +- CometProject (6) + : +- CometFilter (5) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) + +- ReusedExchange (11) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -38,105 +38,104 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(4) CometFilter +(5) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(5) CometProject +(6) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(6) CometBroadcastExchange +(7) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(8) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] +(10) Project [codegen id : 3] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] +(11) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#14] -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None -(14) CometProject +(13) Project [codegen id : 3] +Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(15) CometHashAggregate +(14) HashAggregate [codegen id : 3] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#15] +Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] + +(15) CometColumnarExchange +Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometExchange +(16) CometColumnarToRow [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometHashAggregate +(17) HashAggregate [codegen id : 4] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] +Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(18) CometColumnarExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +(20) CometColumnarToRow [codegen id : 5] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +(22) Project [codegen id : 6] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] (23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(25) CometColumnarToRow [codegen id : 7] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== @@ -149,18 +148,18 @@ BroadcastExchange (30) (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] +Output [2]: [d_date_sk#14, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct (27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) (28) CometProject -Input [2]: [d_date_sk#14, d_date#15] +Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] (29) CometColumnarToRow [codegen id : 1] @@ -168,6 +167,6 @@ Input [1]: [d_date_sk#14] (30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt index 54a4d3c1b7..58cabe9418 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt @@ -5,29 +5,33 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- 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 +Comet accelerated 16 out of 28 eligible operators (57%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 46191f59cd..788b3f8f4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -1,39 +1,47 @@ -WholeStageCodegen (3) +WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) + WholeStageCodegen (6) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (1) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 From 5c529715f82c5ac5b30fd18ffd2cc9872e62efef Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 09:02:01 -0700 Subject: [PATCH 06/10] revert --- .../q10a.native_iceberg_compat/explain.txt | 368 +- .../q10a.native_iceberg_compat/extended.txt | 117 +- .../q10a.native_iceberg_compat/simplified.txt | 124 +- .../q10a/explain.txt | 368 +- .../q10a/extended.txt | 117 +- .../q10a/simplified.txt | 124 +- .../q11.native_iceberg_compat/explain.txt | 745 ++- .../q11.native_iceberg_compat/extended.txt | 191 +- .../q11.native_iceberg_compat/simplified.txt | 214 +- .../q11/explain.txt | 745 ++- .../q11/extended.txt | 191 +- .../q11/simplified.txt | 214 +- .../q12.native_iceberg_compat/explain.txt | 191 +- .../q12.native_iceberg_compat/extended.txt | 66 +- .../q12.native_iceberg_compat/simplified.txt | 78 +- .../q12/explain.txt | 191 +- .../q12/extended.txt | 66 +- .../q12/simplified.txt | 78 +- .../q14.native_iceberg_compat/explain.txt | 1098 ++-- .../q14.native_iceberg_compat/extended.txt | 739 ++- .../q14.native_iceberg_compat/simplified.txt | 347 +- .../q14/explain.txt | 1098 ++-- .../q14/extended.txt | 739 ++- .../q14/simplified.txt | 347 +- .../q14a.native_iceberg_compat/explain.txt | 1520 +++-- .../q14a.native_iceberg_compat/extended.txt | 5115 ++++++++--------- .../q14a.native_iceberg_compat/simplified.txt | 447 +- .../q14a/explain.txt | 1520 +++-- .../q14a/extended.txt | 5115 ++++++++--------- .../q14a/simplified.txt | 447 +- .../q18a.native_iceberg_compat/explain.txt | 1380 ++--- .../q18a.native_iceberg_compat/extended.txt | 463 +- .../q18a.native_iceberg_compat/simplified.txt | 395 +- .../q18a/explain.txt | 1380 ++--- .../q18a/extended.txt | 463 +- .../q18a/simplified.txt | 395 +- .../q20.native_iceberg_compat/explain.txt | 191 +- .../q20.native_iceberg_compat/extended.txt | 66 +- .../q20.native_iceberg_compat/simplified.txt | 78 +- .../q20/explain.txt | 191 +- .../q20/extended.txt | 66 +- .../q20/simplified.txt | 78 +- .../q22.native_iceberg_compat/explain.txt | 201 +- .../q22.native_iceberg_compat/extended.txt | 70 +- .../q22.native_iceberg_compat/simplified.txt | 86 +- .../q22/explain.txt | 201 +- .../q22/extended.txt | 70 +- .../q22/simplified.txt | 86 +- .../q22a.native_iceberg_compat/explain.txt | 511 +- .../q22a.native_iceberg_compat/extended.txt | 342 +- .../q22a.native_iceberg_compat/simplified.txt | 141 +- .../q22a/explain.txt | 511 +- .../q22a/extended.txt | 342 +- .../q22a/simplified.txt | 141 +- .../q24.native_iceberg_compat/explain.txt | 400 +- .../q24.native_iceberg_compat/extended.txt | 120 +- .../q24.native_iceberg_compat/simplified.txt | 102 +- .../q24/explain.txt | 400 +- .../q24/extended.txt | 120 +- .../q24/simplified.txt | 102 +- .../q27a.native_iceberg_compat/explain.txt | 672 +-- .../q27a.native_iceberg_compat/extended.txt | 211 +- .../q27a.native_iceberg_compat/simplified.txt | 199 +- .../q27a/explain.txt | 672 +-- .../q27a/extended.txt | 211 +- .../q27a/simplified.txt | 199 +- .../q34.native_iceberg_compat/explain.txt | 267 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q34.native_iceberg_compat/simplified.txt | 92 +- .../q34/explain.txt | 267 +- .../q34/extended.txt | 82 +- .../q34/simplified.txt | 92 +- .../q35.native_iceberg_compat/explain.txt | 365 +- .../q35.native_iceberg_compat/extended.txt | 130 +- .../q35.native_iceberg_compat/simplified.txt | 142 +- .../q35/explain.txt | 365 +- .../q35/extended.txt | 130 +- .../q35/simplified.txt | 142 +- .../q35a.native_iceberg_compat/explain.txt | 372 +- .../q35a.native_iceberg_compat/extended.txt | 117 +- .../q35a.native_iceberg_compat/simplified.txt | 124 +- .../q35a/explain.txt | 372 +- .../q35a/extended.txt | 117 +- .../q35a/simplified.txt | 124 +- .../q36a.native_iceberg_compat/explain.txt | 429 +- .../q36a.native_iceberg_compat/extended.txt | 223 +- .../q36a.native_iceberg_compat/simplified.txt | 134 +- .../q36a/explain.txt | 429 +- .../q36a/extended.txt | 223 +- .../q36a/simplified.txt | 134 +- .../q47.native_iceberg_compat/explain.txt | 377 +- .../q47.native_iceberg_compat/extended.txt | 225 +- .../q47.native_iceberg_compat/simplified.txt | 150 +- .../q47/explain.txt | 377 +- .../q47/extended.txt | 225 +- .../q47/simplified.txt | 150 +- .../q49.native_iceberg_compat/explain.txt | 713 ++- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q49.native_iceberg_compat/simplified.txt | 168 +- .../q49/explain.txt | 713 ++- .../q49/extended.txt | 144 +- .../q49/simplified.txt | 168 +- .../q51a.native_iceberg_compat/explain.txt | 687 ++- .../q51a.native_iceberg_compat/extended.txt | 456 +- .../q51a.native_iceberg_compat/simplified.txt | 229 +- .../q51a/explain.txt | 687 ++- .../q51a/extended.txt | 456 +- .../q51a/simplified.txt | 229 +- .../q57.native_iceberg_compat/explain.txt | 377 +- .../q57.native_iceberg_compat/extended.txt | 225 +- .../q57.native_iceberg_compat/simplified.txt | 150 +- .../q57/explain.txt | 377 +- .../q57/extended.txt | 225 +- .../q57/simplified.txt | 150 +- .../q5a.native_iceberg_compat/explain.txt | 810 ++- .../q5a.native_iceberg_compat/extended.txt | 571 +- .../q5a.native_iceberg_compat/simplified.txt | 248 +- .../q5a/explain.txt | 810 ++- .../q5a/extended.txt | 571 +- .../q5a/simplified.txt | 248 +- .../q6.native_iceberg_compat/explain.txt | 392 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q6.native_iceberg_compat/simplified.txt | 132 +- .../q6/explain.txt | 392 +- .../q6/extended.txt | 123 +- .../q6/simplified.txt | 132 +- .../q64.native_iceberg_compat/explain.txt | 1426 +++-- .../q64.native_iceberg_compat/extended.txt | 514 +- .../q64.native_iceberg_compat/simplified.txt | 441 +- .../q64/explain.txt | 1426 +++-- .../q64/extended.txt | 514 +- .../q64/simplified.txt | 441 +- .../q67a.native_iceberg_compat/explain.txt | 849 ++- .../q67a.native_iceberg_compat/extended.txt | 648 +-- .../q67a.native_iceberg_compat/simplified.txt | 239 +- .../q67a/explain.txt | 849 ++- .../q67a/extended.txt | 648 +-- .../q67a/simplified.txt | 239 +- .../q70a.native_iceberg_compat/explain.txt | 622 +- .../q70a.native_iceberg_compat/extended.txt | 362 +- .../q70a.native_iceberg_compat/simplified.txt | 206 +- .../q70a/explain.txt | 622 +- .../q70a/extended.txt | 362 +- .../q70a/simplified.txt | 206 +- .../q72.native_iceberg_compat/explain.txt | 492 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q72.native_iceberg_compat/simplified.txt | 155 +- .../q72/explain.txt | 492 +- .../q72/extended.txt | 126 +- .../q72/simplified.txt | 155 +- .../q74.native_iceberg_compat/explain.txt | 745 ++- .../q74.native_iceberg_compat/extended.txt | 191 +- .../q74.native_iceberg_compat/simplified.txt | 214 +- .../q74/explain.txt | 745 ++- .../q74/extended.txt | 191 +- .../q74/simplified.txt | 214 +- .../q75.native_iceberg_compat/explain.txt | 680 ++- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q75.native_iceberg_compat/simplified.txt | 207 +- .../q75/explain.txt | 680 ++- .../q75/extended.txt | 216 +- .../q75/simplified.txt | 207 +- .../q77a.native_iceberg_compat/explain.txt | 1019 ++-- .../q77a.native_iceberg_compat/extended.txt | 686 +-- .../q77a.native_iceberg_compat/simplified.txt | 251 +- .../q77a/explain.txt | 1019 ++-- .../q77a/extended.txt | 686 +-- .../q77a/simplified.txt | 251 +- .../q78.native_iceberg_compat/explain.txt | 525 +- .../q78.native_iceberg_compat/extended.txt | 168 +- .../q78.native_iceberg_compat/simplified.txt | 178 +- .../q78/explain.txt | 525 +- .../q78/extended.txt | 168 +- .../q78/simplified.txt | 178 +- .../q80a.native_iceberg_compat/explain.txt | 1069 ++-- .../q80a.native_iceberg_compat/extended.txt | 829 ++- .../q80a.native_iceberg_compat/simplified.txt | 309 +- .../q80a/explain.txt | 1069 ++-- .../q80a/extended.txt | 829 ++- .../q80a/simplified.txt | 309 +- .../q86a.native_iceberg_compat/explain.txt | 376 +- .../q86a.native_iceberg_compat/extended.txt | 184 +- .../q86a.native_iceberg_compat/simplified.txt | 122 +- .../q86a/explain.txt | 376 +- .../q86a/extended.txt | 184 +- .../q86a/simplified.txt | 122 +- .../q98.native_iceberg_compat/explain.txt | 159 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q98.native_iceberg_compat/simplified.txt | 66 +- .../q98/explain.txt | 159 +- .../q98/extended.txt | 54 +- .../q98/simplified.txt | 66 +- 192 files changed, 36580 insertions(+), 43360 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt index 5552d071da..b6c8291e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt @@ -1,49 +1,44 @@ == Physical Plan == -* CometColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- RowToColumnar (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * CometColumnarToRow (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * CometColumnarToRow (17) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -57,233 +52,216 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customer_sk#14] +Child 1 Input [1]: [customer_sk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customer_sk#14] +Arguments: [customer_sk#14] -(22) BroadcastExchange -Input [1]: [customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customer_sk#14] +Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) - -(27) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(28) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(29) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(27) CometBroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: [ca_address_sk#20] -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ca_address_sk#20] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] +(29) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) +(31) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(34) CometProject -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(32) CometProject +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(35) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(33) CometBroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(36) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(34) CometBroadcastHashJoin +Left output [1]: [c_current_cdemo_sk#2] +Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(38) Project [codegen id : 9] -Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) CometHashAggregate +Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(40) CometColumnarExchange -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +(37) CometExchange +Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] +Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(38) CometHashAggregate +Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] +Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] -(43) RowToColumnar -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +(39) CometTakeOrderedAndProject +Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -(44) CometTakeOrderedAndProject -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_education_status#31 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#32 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#29,cd_marital_status#30,cd_education_status#31,cnt1#36,cd_purchase_estimate#24,cnt2#37,cd_credit_rating#32,cnt3#38,cd_dep_count#26,cnt4#39,cd_dep_employed_count#27,cnt5#40,cd_dep_college_count#28,cnt6#41]), [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41], 100, 0, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] - -(45) CometColumnarToRow [codegen id : 11] -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +(40) CometColumnarToRow [codegen id : 1] +Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#42, d_moy#43] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#7, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 7)) AND isnotnull(d_date_sk#7)) +(42) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(48) CometProject -Input [3]: [d_date_sk#7, d_year#42, d_moy#43] +(43) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(49) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(50) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index fbfef118e0..2cdc75e15e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -1,67 +1,56 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt index 5decf7b86d..38f41d10a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt @@ -1,78 +1,52 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] + CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt index 5552d071da..b6c8291e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt @@ -1,49 +1,44 @@ == Physical Plan == -* CometColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- RowToColumnar (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * CometColumnarToRow (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * CometColumnarToRow (17) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -57,233 +52,216 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customer_sk#14] +Child 1 Input [1]: [customer_sk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customer_sk#14] +Arguments: [customer_sk#14] -(22) BroadcastExchange -Input [1]: [customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customer_sk#14] +Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) - -(27) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(28) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(29) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(27) CometBroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: [ca_address_sk#20] -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ca_address_sk#20] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] +(29) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) +(31) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(34) CometProject -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(32) CometProject +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(35) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(33) CometBroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(36) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(34) CometBroadcastHashJoin +Left output [1]: [c_current_cdemo_sk#2] +Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(38) Project [codegen id : 9] -Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) CometHashAggregate +Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(40) CometColumnarExchange -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] +(37) CometExchange +Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] +Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(38) CometHashAggregate +Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] +Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] -(43) RowToColumnar -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +(39) CometTakeOrderedAndProject +Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -(44) CometTakeOrderedAndProject -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_education_status#31 ASC NULLS FIRST,cd_purchase_estimate#24 ASC NULLS FIRST,cd_credit_rating#32 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[cd_gender#29,cd_marital_status#30,cd_education_status#31,cnt1#36,cd_purchase_estimate#24,cnt2#37,cd_credit_rating#32,cnt3#38,cd_dep_count#26,cnt4#39,cd_dep_employed_count#27,cnt5#40,cd_dep_college_count#28,cnt6#41]), [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41], 100, 0, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] - -(45) CometColumnarToRow [codegen id : 11] -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] +(40) CometColumnarToRow [codegen id : 1] +Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#42, d_moy#43] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#7, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 2002)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 7)) AND isnotnull(d_date_sk#7)) +(42) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(48) CometProject -Input [3]: [d_date_sk#7, d_year#42, d_moy#43] +(43) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(49) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(50) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt index fbfef118e0..2cdc75e15e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt @@ -1,67 +1,56 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt index 5decf7b86d..38f41d10a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt @@ -1,78 +1,52 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] + CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [customer_sk] #5 + CometUnion [customer_sk] + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt index 7a08937760..49c1c8e0d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt @@ -1,85 +1,76 @@ == Physical Plan == -* CometColumnarToRow (81) -+- CometTakeOrderedAndProject (80) - +- RowToColumnar (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * CometColumnarToRow (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometFilter (64) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- CometProject (70) + +- CometBroadcastHashJoin (69) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometBroadcastHashJoin (35) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (34) + : : +- CometHashAggregate (33) + : : +- CometExchange (32) + : : +- CometHashAggregate (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (51) + : +- CometFilter (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) + +- CometBroadcastExchange (68) + +- CometHashAggregate (67) + +- CometExchange (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -97,10 +88,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -108,414 +96,375 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#15) -(7) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] +Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +(8) CometProject Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(11) ReusedExchange [Reuses operator id: 85] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#20, d_year#21] +Arguments: [d_date_sk#20, d_year#21] + +(12) CometBroadcastHashJoin +Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Right output [2]: [d_date_sk#20, d_year#21] +Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] +(13) CometProject Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] +(16) CometHashAggregate +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#23, year_total#24] +Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] +(19) CometFilter +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +(20) CometProject +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(25) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] - -(36) BroadcastExchange -Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +(22) CometFilter +Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Condition : isnotnull(ss_customer_sk#33) + +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] + +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight + +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] + +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: [d_date_sk#38, d_year#39] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] + +(31) CometHashAggregate +Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] + +(32) CometExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] + +(34) CometBroadcastExchange +Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) +(37) CometFilter +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) -(40) CometProject -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] +(38) CometProject +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Condition : isnotnull(ws_bill_customer_sk#63) +(40) CometFilter +Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#60) -(44) CometColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +(41) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -(45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] +Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight + +(43) CometProject +Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] + +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#65, d_year#66] + +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Right output [2]: [d_date_sk#65, d_year#66] +Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight + +(46) CometProject +Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] + +(47) CometHashAggregate +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] + +(48) CometExchange +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(49) CometHashAggregate +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] + +(50) CometFilter +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#68, year_total#69] + +(52) CometBroadcastHashJoin +Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Right output [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight + +(53) CometProject +Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] +Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [ws_bill_customer_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(48) ReusedExchange [Reuses operator id: 85] -Output [2]: [d_date_sk#68, d_year#69] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#68] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#68, d_year#69] - -(51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum#70] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] - -(52) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] - -(54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72] -Results [2]: [c_customer_id#57 AS customer_id#73, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72,18,2) AS year_total#74] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#73, year_total#74] -Condition : (isnotnull(year_total#74) AND (year_total#74 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#73] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74] -Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#73, year_total#74] - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(60) CometFilter -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Condition : (isnotnull(c_customer_sk#75) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true))) +(55) CometFilter +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) -(61) CometProject -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Arguments: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62], [c_customer_sk#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#77, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#78, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#79, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#80, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#81, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#82, 50, true, false, true) AS c_email_address#62] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62] +(56) CometProject +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_bill_customer_sk#83) - -(65) CometColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] - -(66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#75] -Right keys [1]: [ws_bill_customer_sk#83] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Input [12]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] - -(69) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#88, d_year#89] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] - -(72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] - -(75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72] -Results [2]: [c_customer_id#57 AS customer_id#92, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72,18,2) AS year_total#93] +(58) CometFilter +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#78) -(76) BroadcastExchange -Input [2]: [customer_id#92, year_total#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(59) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] + +(60) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] +Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight + +(61) CometProject +Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#83, d_year#84] -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#92] -Join type: Inner -Join condition: (CASE WHEN (year_total#74 > 0.00) THEN (year_total#93 / year_total#74) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) +(63) CometBroadcastHashJoin +Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Right output [2]: [d_date_sk#83, d_year#84] +Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight -(78) Project [codegen id : 16] -Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74, customer_id#92, year_total#93] +(64) CometProject +Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -(79) RowToColumnar -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(65) CometHashAggregate +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] -(80) CometTakeOrderedAndProject -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#44 ASC NULLS FIRST,customer_first_name#45 ASC NULLS FIRST,customer_last_name#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#44,customer_first_name#45,customer_last_name#46,customer_email_address#47]), [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47], 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(66) CometExchange +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(81) CometColumnarToRow [codegen id : 17] -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(67) CometHashAggregate +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#86, year_total#87] +Arguments: [customer_id#86, year_total#87] + +(69) CometBroadcastHashJoin +Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] +Right output [2]: [customer_id#86, year_total#87] +Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight + +(70) CometProject +Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] + +(71) CometTakeOrderedAndProject +Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] + +(72) CometColumnarToRow [codegen id : 1] +Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometFilter (83) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(83) CometFilter +(74) CometFilter Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(85) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) +(78) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(88) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#38, d_year#39] -(89) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(80) BroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index f5dda0de66..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -1,107 +1,90 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt index 0e65a6a74f..157d1d587e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt @@ -1,134 +1,90 @@ -WholeStageCodegen (17) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - RowToColumnar - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 + CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt index 7a08937760..49c1c8e0d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt @@ -1,85 +1,76 @@ == Physical Plan == -* CometColumnarToRow (81) -+- CometTakeOrderedAndProject (80) - +- RowToColumnar (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * CometColumnarToRow (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometFilter (64) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- CometProject (70) + +- CometBroadcastHashJoin (69) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometBroadcastHashJoin (35) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (34) + : : +- CometHashAggregate (33) + : : +- CometExchange (32) + : : +- CometHashAggregate (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (51) + : +- CometFilter (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) + +- CometBroadcastExchange (68) + +- CometHashAggregate (67) + +- CometExchange (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -97,10 +88,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -108,414 +96,375 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Condition : isnotnull(ss_customer_sk#15) -(7) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] +Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +(8) CometProject Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -(11) ReusedExchange [Reuses operator id: 85] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#20, d_year#21] +Arguments: [d_date_sk#20, d_year#21] + +(12) CometBroadcastHashJoin +Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] +Right output [2]: [d_date_sk#20, d_year#21] +Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] +(13) CometProject Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] +(16) CometHashAggregate +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#23, year_total#24] +Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] +(19) CometFilter +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] +(20) CometProject +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(25) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] - -(36) BroadcastExchange -Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] +(22) CometFilter +Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Condition : isnotnull(ss_customer_sk#33) + +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] + +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight + +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] + +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: [d_date_sk#38, d_year#39] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] +Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] + +(31) CometHashAggregate +Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] + +(32) CometExchange +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] +Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] +Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] + +(34) CometBroadcastExchange +Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) +(37) CometFilter +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) -(40) CometProject -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] +(38) CometProject +Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] +Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Condition : isnotnull(ws_bill_customer_sk#63) +(40) CometFilter +Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#60) -(44) CometColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] +(41) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -(45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] +Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight + +(43) CometProject +Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] + +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#65, d_year#66] + +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] +Right output [2]: [d_date_sk#65, d_year#66] +Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight + +(46) CometProject +Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] + +(47) CometHashAggregate +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] + +(48) CometExchange +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(49) CometHashAggregate +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] + +(50) CometFilter +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#68, year_total#69] + +(52) CometBroadcastHashJoin +Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] +Right output [2]: [customer_id#68, year_total#69] +Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight + +(53) CometProject +Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] +Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [ws_bill_customer_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(48) ReusedExchange [Reuses operator id: 85] -Output [2]: [d_date_sk#68, d_year#69] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#68] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#68, d_year#69] - -(51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#69] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum#70] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] - -(52) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] - -(54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69, sum#71] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#69] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72] -Results [2]: [c_customer_id#57 AS customer_id#73, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#72,18,2) AS year_total#74] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#73, year_total#74] -Condition : (isnotnull(year_total#74) AND (year_total#74 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#73] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74] -Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#73, year_total#74] - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(60) CometFilter -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Condition : (isnotnull(c_customer_sk#75) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true))) +(55) CometFilter +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) -(61) CometProject -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Arguments: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62], [c_customer_sk#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#76, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#77, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#78, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#79, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#80, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#81, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#82, 50, true, false, true) AS c_email_address#62] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62] +(56) CometProject +Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] +Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_bill_customer_sk#83) - -(65) CometColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] - -(66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#75] -Right keys [1]: [ws_bill_customer_sk#83] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Input [12]: [c_customer_sk#75, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] - -(69) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#88, d_year#89] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] - -(72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] - -(75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89, sum#91] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#80, c_login#61, c_email_address#62, d_year#89] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72] -Results [2]: [c_customer_id#57 AS customer_id#92, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#85 - ws_ext_discount_amt#84)))#72,18,2) AS year_total#93] +(58) CometFilter +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#78) -(76) BroadcastExchange -Input [2]: [customer_id#92, year_total#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(59) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] + +(60) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] +Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight + +(61) CometProject +Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#83, d_year#84] -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#92] -Join type: Inner -Join condition: (CASE WHEN (year_total#74 > 0.00) THEN (year_total#93 / year_total#74) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) +(63) CometBroadcastHashJoin +Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Right output [2]: [d_date_sk#83, d_year#84] +Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight -(78) Project [codegen id : 16] -Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#74, customer_id#92, year_total#93] +(64) CometProject +Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] +Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -(79) RowToColumnar -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(65) CometHashAggregate +Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] -(80) CometTakeOrderedAndProject -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#44 ASC NULLS FIRST,customer_first_name#45 ASC NULLS FIRST,customer_last_name#46 ASC NULLS FIRST,customer_email_address#47 ASC NULLS FIRST], output=[customer_id#44,customer_first_name#45,customer_last_name#46,customer_email_address#47]), [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47], 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(66) CometExchange +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] +Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(81) CometColumnarToRow [codegen id : 17] -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +(67) CometHashAggregate +Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] +Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#86, year_total#87] +Arguments: [customer_id#86, year_total#87] + +(69) CometBroadcastHashJoin +Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] +Right output [2]: [customer_id#86, year_total#87] +Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight + +(70) CometProject +Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] +Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] + +(71) CometTakeOrderedAndProject +Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] + +(72) CometColumnarToRow [codegen id : 1] +Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometFilter (83) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#20, d_year#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(83) CometFilter +(74) CometFilter Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(85) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) +(78) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(88) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#38, d_year#39] -(89) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(80) BroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt index f5dda0de66..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt @@ -1,107 +1,90 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt index 0e65a6a74f..157d1d587e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt @@ -1,134 +1,90 @@ -WholeStageCodegen (17) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - RowToColumnar - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 + CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt index 58d7150a81..410aecff1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt @@ -1,29 +1,27 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- RowToColumnar (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -38,134 +36,129 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(15) CometColumnarExchange +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) RowToColumnar -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(24) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(25) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +(26) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 9c95a71e3f..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -1,37 +1,31 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt index c4ddd74416..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt @@ -1,47 +1,35 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - RowToColumnar - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt index 58d7150a81..410aecff1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt @@ -1,29 +1,27 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- RowToColumnar (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -38,134 +36,129 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(15) CometColumnarExchange +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) RowToColumnar -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(24) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(25) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +(26) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt index 9c95a71e3f..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt @@ -1,37 +1,31 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt index c4ddd74416..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt @@ -1,47 +1,35 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - RowToColumnar - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt index 5f8639ff9f..ea550c6c74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt @@ -1,93 +1,89 @@ == Physical Plan == -* CometColumnarToRow (89) -+- CometTakeOrderedAndProject (88) - +- RowToColumnar (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * CometColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * CometColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * CometColumnarToRow (39) - : : : : : +- CometFilter (38) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * CometColumnarToRow (71) - : : : +- CometFilter (70) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) +* CometColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -102,24 +98,18 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -127,28 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) CometColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(8) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -156,622 +140,604 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) CometColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(12) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(13) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(15) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) -(22) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#24] +(18) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(21) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(23) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(26) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#25] +(27) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(29) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(30) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(31) CometExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(35) CometHashAggregate -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) - -(39) CometColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(43) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#36] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] - -(46) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(34) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) + +(35) ReusedExchange [Reuses operator id: 13] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] + +(36) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight + +(37) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] + +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#37] -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(39) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(40) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] + +(41) CometBroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] + +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight + +(43) CometBroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [brand_id#27, class_id#28, category_id#29] + +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight + +(45) CometProject +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] + +(46) CometBroadcastExchange +Input [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#38] + +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight + +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +(49) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(50) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#38] -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#37] +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(52) CometBroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None +(54) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) ReusedExchange [Reuses operator id: 114] -Output [1]: [d_date_sk#42] +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(59) CometBroadcastHashJoin +Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(65) CometColumnarExchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(61) CometHashAggregate +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(62) CometExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(68) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) - -(71) CometColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(77) ReusedExchange [Reuses operator id: 128] -Output [1]: [d_date_sk#66] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] -Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] - -(84) Filter [codegen id : 51] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Join type: Inner -Join condition: None - -(87) RowToColumnar -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] - -(88) CometTakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sales#52,number_sales#53,channel#75,i_brand_id#63,i_class_id#64,i_category_id#65,sales#76,number_sales#77]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77], 100, 0, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] - -(89) CometColumnarToRow [codegen id : 53] -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#60] + +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#60] +Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] + +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) + +(74) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: [d_date_sk#65] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometExchange +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(80) CometHashAggregate +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(85) CometColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (109) -+- * CometColumnarToRow (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * CometColumnarToRow (91) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (90) - : +- ReusedExchange (92) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * CometColumnarToRow (96) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (95) - : +- ReusedExchange (97) - +- * Project (104) - +- * BroadcastHashJoin Inner BuildRight (103) - :- * CometColumnarToRow (101) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (100) - +- ReusedExchange (102) - - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* CometColumnarToRow (102) ++- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] ReadSchema: struct -(91) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] - -(92) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#82] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#79] -(93) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#80] -Right keys [1]: [d_date_sk#82] -Join type: Inner -Join condition: None +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Right output [1]: [d_date_sk#79] +Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight -(94) Project [codegen id : 2] -Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +(89) CometProject +Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] +Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] ReadSchema: struct -(96) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] - -(97) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#89] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#86] -(98) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#87] -Right keys [1]: [d_date_sk#89] -Join type: Inner -Join condition: None +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight -(99) Project [codegen id : 4] -Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +(93) CometProject +Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] +Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] -(100) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] ReadSchema: struct -(101) CometColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#93] -(102) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#96] +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +Right output [1]: [d_date_sk#93] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight -(103) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#96] -Join type: Inner -Join condition: None +(97) CometProject +Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] +Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] -(104) Project [codegen id : 6] -Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +(98) CometUnion +Child 0 Input [2]: [quantity#80, list_price#81] +Child 1 Input [2]: [quantity#87, list_price#88] +Child 2 Input [2]: [quantity#94, list_price#95] -(105) Union - -(106) HashAggregate [codegen id : 7] -Input [2]: [quantity#83, list_price#84] +(99) CometHashAggregate +Input [2]: [quantity#80, list_price#81] Keys: [] -Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [2]: [sum#99, count#100] -Results [2]: [sum#101, count#102] - -(107) CometColumnarExchange -Input [2]: [sum#101, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(108) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#101, count#102] +(100) CometExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(109) HashAggregate [codegen id : 8] -Input [2]: [sum#101, count#102] +(101) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] +Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +(102) CometColumnarToRow [codegen id : 1] +Input [1]: [average_sales#98] -Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (114) -+- * CometColumnarToRow (113) - +- CometProject (112) - +- CometFilter (111) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (110) +BroadcastExchange (107) ++- * CometColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#105] +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(111) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#105] -Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(112) CometProject -Input [2]: [d_date_sk#42, d_week_seq#105] -Arguments: [d_date_sk#42], [d_date_sk#42] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(113) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(106) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(114) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(107) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#106, [id=#107] -* CometColumnarToRow (118) -+- CometProject (117) - +- CometFilter (116) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (115) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* CometColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) -(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(116) CometFilter -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) +(109) CometFilter +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) -(117) CometProject -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Arguments: [d_week_seq#108], [d_week_seq#108] +(110) CometProject +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Arguments: [d_week_seq#99], [d_week_seq#99] -(118) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#108] +(111) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#99] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (123) -+- * CometColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (119) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * CometColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#112] +(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter -Input [2]: [d_date_sk#25, d_year#112] -Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) + +(114) CometProject +Input [2]: [d_date_sk#26, d_year#103] +Arguments: [d_date_sk#26], [d_date_sk#26] -(121) CometProject -Input [2]: [d_date_sk#25, d_year#112] -Arguments: [d_date_sk#25], [d_date_sk#25] +(115) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(122) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(116) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(123) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometProject (126) - +- CometFilter (125) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (124) +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(124) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#113] +(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(125) CometFilter -Input [2]: [d_date_sk#66, d_week_seq#113] -Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) +(118) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) -(126) CometProject -Input [2]: [d_date_sk#66, d_week_seq#113] -Arguments: [d_date_sk#66], [d_date_sk#66] +(119) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] -(127) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(120) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] -(128) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(121) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#114, [id=#115] -* CometColumnarToRow (132) -+- CometProject (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +* CometColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(130) CometFilter -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +(123) CometFilter +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) + +(124) CometProject +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Arguments: [d_week_seq#104], [d_week_seq#104] -(131) CometProject -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Arguments: [d_week_seq#116], [d_week_seq#116] +(125) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#104] -(132) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#116] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index 88a9611f1c..f56d229b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -1,406 +1,339 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 -Comet accelerated 160 out of 333 eligible operators (48%). Final plan contains 71 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt index b038a65567..fb9abae378 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt @@ -1,210 +1,153 @@ -WholeStageCodegen (53) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - RowToColumnar - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] + CometExchange #14 + CometHashAggregate [quantity,list_price] [sum,count] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometExchange [brand_id,class_id,category_id] #5 + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt index 5f8639ff9f..ea550c6c74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt @@ -1,93 +1,89 @@ == Physical Plan == -* CometColumnarToRow (89) -+- CometTakeOrderedAndProject (88) - +- RowToColumnar (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * CometColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * CometColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * CometColumnarToRow (39) - : : : : : +- CometFilter (38) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * CometColumnarToRow (71) - : : : +- CometFilter (70) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) +* CometColumnarToRow (85) ++- CometTakeOrderedAndProject (84) + +- CometBroadcastHashJoin (83) + :- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometExchange (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometBroadcastHashJoin (47) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometHashAggregate (32) + : : : : +- CometExchange (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometProject (21) + : : : : : +- CometBroadcastHashJoin (20) + : : : : : :- CometProject (15) + : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) + : : : : : : +- CometBroadcastExchange (13) + : : : : : : +- CometFilter (12) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) + : : : : : +- CometBroadcastExchange (19) + : : : : : +- CometProject (18) + : : : : : +- CometFilter (17) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) + : : : : +- ReusedExchange (27) + : : : +- CometBroadcastExchange (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (34) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) + : : : : +- ReusedExchange (35) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometFilter (49) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) + : : +- ReusedExchange (50) + : +- CometBroadcastExchange (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + +- CometBroadcastExchange (82) + +- CometFilter (81) + +- CometHashAggregate (80) + +- CometExchange (79) + +- CometHashAggregate (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometBroadcastHashJoin (68) + : : :- CometFilter (66) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) + : : +- ReusedExchange (67) + : +- ReusedExchange (69) + +- CometBroadcastExchange (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -102,24 +98,18 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -127,28 +117,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) CometColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(8) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -156,622 +140,604 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) CometColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(12) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(13) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(15) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) -(22) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#24] +(18) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(21) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(23) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(26) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#25] +(27) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(29) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(30) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(31) CometExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(35) CometHashAggregate -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) - -(39) CometColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(43) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#36] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] - -(46) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(34) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) + +(35) ReusedExchange [Reuses operator id: 13] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] + +(36) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight + +(37) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] + +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#37] -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(39) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(40) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] + +(41) CometBroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] + +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight + +(43) CometBroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [brand_id#27, class_id#28, category_id#29] + +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight + +(45) CometProject +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] + +(46) CometBroadcastExchange +Input [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#38] + +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight + +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +(49) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(50) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#38] -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#37] +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(52) CometBroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None +(54) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) ReusedExchange [Reuses operator id: 114] -Output [1]: [d_date_sk#42] +(56) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(57) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(59) CometBroadcastHashJoin +Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight + +(60) CometProject +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -(65) CometColumnarExchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(61) CometHashAggregate +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(62) CometExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(68) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(64) CometFilter +Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) - -(71) CometColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(77) ReusedExchange [Reuses operator id: 128] -Output [1]: [d_date_sk#66] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] -Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] - -(84) Filter [codegen id : 51] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Join type: Inner -Join condition: None - -(87) RowToColumnar -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] - -(88) CometTakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sales#52,number_sales#53,channel#75,i_brand_id#63,i_class_id#64,i_category_id#65,sales#76,number_sales#77]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77], 100, 0, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] - -(89) CometColumnarToRow [codegen id : 53] -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +(66) CometFilter +Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Condition : isnotnull(ss_item_sk#55) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#60] + +(68) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [1]: [ss_item_sk#60] +Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] + +(70) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] +Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight + +(71) CometProject +Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] + +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) + +(74) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(75) CometBroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: [d_date_sk#65] + +(76) CometBroadcastHashJoin +Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(77) CometProject +Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] + +(78) CometHashAggregate +Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] + +(79) CometExchange +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(80) CometHashAggregate +Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] + +(81) CometFilter +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(82) CometBroadcastExchange +Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(83) CometBroadcastHashJoin +Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] +Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight + +(84) CometTakeOrderedAndProject +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] + +(85) CometColumnarToRow [codegen id : 1] +Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (109) -+- * CometColumnarToRow (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * CometColumnarToRow (91) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (90) - : +- ReusedExchange (92) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * CometColumnarToRow (96) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (95) - : +- ReusedExchange (97) - +- * Project (104) - +- * BroadcastHashJoin Inner BuildRight (103) - :- * CometColumnarToRow (101) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (100) - +- ReusedExchange (102) - - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* CometColumnarToRow (102) ++- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometUnion (98) + :- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) + : +- ReusedExchange (87) + :- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (91) + +- CometProject (97) + +- CometBroadcastHashJoin (96) + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) + +- ReusedExchange (95) + + +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] ReadSchema: struct -(91) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] - -(92) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#82] +(87) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#79] -(93) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#80] -Right keys [1]: [d_date_sk#82] -Join type: Inner -Join condition: None +(88) CometBroadcastHashJoin +Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] +Right output [1]: [d_date_sk#79] +Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight -(94) Project [codegen id : 2] -Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +(89) CometProject +Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] +Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] ReadSchema: struct -(96) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] - -(97) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#89] +(91) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#86] -(98) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#87] -Right keys [1]: [d_date_sk#89] -Join type: Inner -Join condition: None +(92) CometBroadcastHashJoin +Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight -(99) Project [codegen id : 4] -Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +(93) CometProject +Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] +Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] -(100) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] ReadSchema: struct -(101) CometColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(95) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#93] -(102) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#96] +(96) CometBroadcastHashJoin +Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] +Right output [1]: [d_date_sk#93] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight -(103) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#96] -Join type: Inner -Join condition: None +(97) CometProject +Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] +Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] -(104) Project [codegen id : 6] -Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +(98) CometUnion +Child 0 Input [2]: [quantity#80, list_price#81] +Child 1 Input [2]: [quantity#87, list_price#88] +Child 2 Input [2]: [quantity#94, list_price#95] -(105) Union - -(106) HashAggregate [codegen id : 7] -Input [2]: [quantity#83, list_price#84] +(99) CometHashAggregate +Input [2]: [quantity#80, list_price#81] Keys: [] -Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [2]: [sum#99, count#100] -Results [2]: [sum#101, count#102] - -(107) CometColumnarExchange -Input [2]: [sum#101, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(108) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#101, count#102] +(100) CometExchange +Input [2]: [sum#96, count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(109) HashAggregate [codegen id : 8] -Input [2]: [sum#101, count#102] +(101) CometHashAggregate +Input [2]: [sum#96, count#97] Keys: [] -Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] +Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Subquery:2 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +(102) CometColumnarToRow [codegen id : 1] +Input [1]: [average_sales#98] -Subquery:3 Hosting operator id = 95 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 100 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (114) -+- * CometColumnarToRow (113) - +- CometProject (112) - +- CometFilter (111) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (110) +BroadcastExchange (107) ++- * CometColumnarToRow (106) + +- CometProject (105) + +- CometFilter (104) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#105] +(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(111) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#105] -Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) +(104) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) -(112) CometProject -Input [2]: [d_date_sk#42, d_week_seq#105] -Arguments: [d_date_sk#42], [d_date_sk#42] +(105) CometProject +Input [2]: [d_date_sk#43, d_week_seq#44] +Arguments: [d_date_sk#43], [d_date_sk#43] -(113) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(106) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(114) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(107) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:6 Hosting operator id = 111 Hosting Expression = Subquery scalar-subquery#106, [id=#107] -* CometColumnarToRow (118) -+- CometProject (117) - +- CometFilter (116) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (115) +Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* CometColumnarToRow (111) ++- CometProject (110) + +- CometFilter (109) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) -(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(116) CometFilter -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) +(109) CometFilter +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) -(117) CometProject -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Arguments: [d_week_seq#108], [d_week_seq#108] +(110) CometProject +Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] +Arguments: [d_week_seq#99], [d_week_seq#99] -(118) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#108] +(111) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#99] -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (123) -+- * CometColumnarToRow (122) - +- CometProject (121) - +- CometFilter (120) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (119) +Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (116) ++- * CometColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#112] +(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(120) CometFilter -Input [2]: [d_date_sk#25, d_year#112] -Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) +(113) CometFilter +Input [2]: [d_date_sk#26, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) + +(114) CometProject +Input [2]: [d_date_sk#26, d_year#103] +Arguments: [d_date_sk#26], [d_date_sk#26] -(121) CometProject -Input [2]: [d_date_sk#25, d_year#112] -Arguments: [d_date_sk#25], [d_date_sk#25] +(115) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(122) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(116) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(123) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometProject (126) - +- CometFilter (125) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (124) +Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(124) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#113] +(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_week_seq#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(125) CometFilter -Input [2]: [d_date_sk#66, d_week_seq#113] -Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) +(118) CometFilter +Input [2]: [d_date_sk#65, d_week_seq#66] +Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) -(126) CometProject -Input [2]: [d_date_sk#66, d_week_seq#113] -Arguments: [d_date_sk#66], [d_date_sk#66] +(119) CometProject +Input [2]: [d_date_sk#65, d_week_seq#66] +Arguments: [d_date_sk#65], [d_date_sk#65] -(127) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(120) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] -(128) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(121) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:12 Hosting operator id = 125 Hosting Expression = Subquery scalar-subquery#114, [id=#115] -* CometColumnarToRow (132) -+- CometProject (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) +Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +* CometColumnarToRow (125) ++- CometProject (124) + +- CometFilter (123) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(130) CometFilter -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +(123) CometFilter +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) + +(124) CometProject +Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] +Arguments: [d_week_seq#104], [d_week_seq#104] -(131) CometProject -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Arguments: [d_week_seq#116], [d_week_seq#116] +(125) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#104] -(132) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#116] +Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt index 88a9611f1c..f56d229b68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt @@ -1,406 +1,339 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 -Comet accelerated 160 out of 333 eligible operators (48%). Final plan contains 71 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt index b038a65567..fb9abae378 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt @@ -1,210 +1,153 @@ -WholeStageCodegen (53) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - RowToColumnar - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] + CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] + CometExchange #14 + CometHashAggregate [quantity,list_price] [sum,count] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #1 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [ss_item_sk] #3 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [brand_id,class_id,category_id] #4 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometExchange [brand_id,class_id,category_id] #5 + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [d_date_sk] #10 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #3 + CometBroadcastExchange [d_date_sk] #13 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #4 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #16 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [ss_item_sk] #3 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt index 7d029ef698..a727c5e743 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt @@ -1,143 +1,126 @@ == Physical Plan == -* CometColumnarToRow (139) -+- CometTakeOrderedAndProject (138) - +- CometHashAggregate (137) - +- CometColumnarExchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (105) - : +- * CometColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- * HashAggregate (102) - : +- Union (101) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) - : : +- CometColumnarExchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometColumnarExchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * CometColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * CometColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * CometColumnarToRow (39) - : : : : : : +- CometFilter (38) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (58) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (61) - : :- * Filter (84) - : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) - : : +- CometColumnarExchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * CometColumnarToRow (71) - : : : : : +- CometFilter (70) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometColumnarExchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : : :- * CometColumnarToRow (87) - : : : : +- CometFilter (86) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (85) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- CometColumnarExchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) - : +- ReusedExchange (106) - :- * HashAggregate (119) - : +- * CometColumnarToRow (118) - : +- CometColumnarExchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (126) - : +- * CometColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- * HashAggregate (123) - : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (133) - +- * CometColumnarToRow (132) - +- CometColumnarExchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- * CometColumnarToRow (128) - +- ReusedExchange (127) +* CometColumnarToRow (122) ++- CometTakeOrderedAndProject (121) + +- CometHashAggregate (120) + +- CometExchange (119) + +- CometHashAggregate (118) + +- CometUnion (117) + :- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometUnion (93) + : :- CometFilter (64) + : : +- CometHashAggregate (63) + : : +- CometExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastHashJoin (47) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (46) + : : : : +- CometProject (45) + : : : : +- CometBroadcastHashJoin (44) + : : : : :- CometFilter (4) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : : +- CometBroadcastExchange (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometHashAggregate (32) + : : : : : +- CometExchange (31) + : : : : : +- CometHashAggregate (30) + : : : : : +- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (6) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometBroadcastHashJoin (23) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : : : : : : +- CometBroadcastExchange (22) + : : : : : : +- CometProject (21) + : : : : : : +- CometBroadcastHashJoin (20) + : : : : : : :- CometProject (15) + : : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : : :- CometFilter (10) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) + : : : : : : : +- CometBroadcastExchange (13) + : : : : : : : +- CometFilter (12) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) + : : : : : : +- CometBroadcastExchange (19) + : : : : : : +- CometProject (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) + : : : : : +- ReusedExchange (27) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometProject (37) + : : : : : +- CometBroadcastHashJoin (36) + : : : : : :- CometFilter (34) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) + : : : : : +- ReusedExchange (35) + : : : : +- ReusedExchange (38) + : : : +- CometBroadcastExchange (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) + : : : +- ReusedExchange (50) + : : +- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + : :- CometFilter (78) + : : +- CometHashAggregate (77) + : : +- CometExchange (76) + : : +- CometHashAggregate (75) + : : +- CometProject (74) + : : +- CometBroadcastHashJoin (73) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometBroadcastHashJoin (68) + : : : : :- CometFilter (66) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) + : : : : +- ReusedExchange (67) + : : : +- ReusedExchange (69) + : : +- ReusedExchange (72) + : +- CometFilter (92) + : +- CometHashAggregate (91) + : +- CometExchange (90) + : +- CometHashAggregate (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometBroadcastHashJoin (82) + : : : :- CometFilter (80) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) + : : : +- ReusedExchange (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- CometHashAggregate (101) + : +- CometExchange (100) + : +- CometHashAggregate (99) + : +- CometHashAggregate (98) + : +- ReusedExchange (97) + :- CometHashAggregate (106) + : +- CometExchange (105) + : +- CometHashAggregate (104) + : +- CometHashAggregate (103) + : +- ReusedExchange (102) + :- CometHashAggregate (111) + : +- CometExchange (110) + : +- CometHashAggregate (109) + : +- CometHashAggregate (108) + : +- ReusedExchange (107) + +- CometHashAggregate (116) + +- CometExchange (115) + +- CometHashAggregate (114) + +- CometHashAggregate (113) + +- ReusedExchange (112) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -152,24 +135,18 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -177,28 +154,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) CometColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(8) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -206,815 +177,738 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) CometColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(12) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) CometColumnarToRow [codegen id : 1] +(13) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) BroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(14) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(15) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) + +(18) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] -(22) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#24] +(19) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(21) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(23) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(26) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#25] +(27) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(29) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(30) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(31) CometExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(35) CometHashAggregate -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) - -(39) CometColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(43) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#36] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] - -(46) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(34) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) + +(35) ReusedExchange [Reuses operator id: 13] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] + +(36) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight + +(37) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] + +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#37] + +(39) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight + +(40) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] + +(41) CometBroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] + +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight + +(43) CometBroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [brand_id#27, class_id#28, category_id#29] + +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight + +(45) CometProject +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] + +(46) CometBroadcastExchange +Input [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#38] + +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight + +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(54) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +(49) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(50) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#38] -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#37] +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(52) CometBroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None +(54) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#42] +(56) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(59) CometBroadcastHashJoin +Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(65) CometColumnarExchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(60) CometProject +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(61) CometHashAggregate +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(62) CometExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(68) Filter [codegen id : 26] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(64) CometFilter +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -Condition : isnotnull(cs_item_sk#56) - -(71) CometColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(75) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 51] -Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(77) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#66] - -(78) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 51] -Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(80) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(83) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] -Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] - -(84) Filter [codegen id : 52] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +(66) CometFilter +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Condition : isnotnull(cs_item_sk#54) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#59] + +(68) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [1]: [ss_item_sk#59] +Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] + +(70) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight + +(71) CometProject +Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] + +(72) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#64] + +(73) CometBroadcastHashJoin +Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] +Right output [1]: [d_date_sk#64] +Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight + +(74) CometProject +Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] +Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] + +(75) CometHashAggregate +Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] + +(76) CometExchange +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(77) CometHashAggregate +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] + +(78) CometFilter +Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) CometFilter -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_item_sk#78) - -(87) CometColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] - -(88) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#83] - -(89) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [ss_item_sk#83] -Join type: LeftSemi -Join condition: None - -(90) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [i_item_sk#84] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 77] -Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] - -(93) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#88] - -(94) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 77] -Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] - -(96) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] - -(97) CometColumnarExchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] - -(99) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] -Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] - -(100) Filter [codegen id : 78] -Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] -Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(101) Union - -(102) HashAggregate [codegen id : 79] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] -Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] - -(103) CometColumnarExchange -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(104) CometColumnarToRow [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] - -(105) HashAggregate [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] - -(106) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] - -(107) CometColumnarToRow [codegen id : 160] -Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] - -(108) HashAggregate [codegen id : 160] -Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [5]: [channel#51, i_brand_id#110, i_class_id#111, sum(sales#52)#106 AS sum_sales#113, sum(number_sales#53)#107 AS number_sales#114] - -(109) HashAggregate [codegen id : 160] -Input [5]: [channel#51, i_brand_id#110, i_class_id#111, sum_sales#113, number_sales#114] -Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] -Functions [2]: [partial_sum(sum_sales#113), partial_sum(number_sales#114)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] - -(110) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#51, i_brand_id#110, i_class_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(111) CometColumnarToRow [codegen id : 161] -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] - -(112) HashAggregate [codegen id : 161] -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] -Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] -Functions [2]: [sum(sum_sales#113), sum(number_sales#114)] -Aggregate Attributes [2]: [sum(sum_sales#113)#121, sum(number_sales#114)#122] -Results [6]: [channel#51, i_brand_id#110, i_class_id#111, null AS i_category_id#123, sum(sum_sales#113)#121 AS sum(sum_sales)#124, sum(number_sales#114)#122 AS sum(number_sales)#125] - -(113) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] - -(114) CometColumnarToRow [codegen id : 241] -Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] - -(115) HashAggregate [codegen id : 241] -Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [4]: [channel#51, i_brand_id#126, sum(sales#52)#106 AS sum_sales#129, sum(number_sales#53)#107 AS number_sales#130] - -(116) HashAggregate [codegen id : 241] -Input [4]: [channel#51, i_brand_id#126, sum_sales#129, number_sales#130] -Keys [2]: [channel#51, i_brand_id#126] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] -Results [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] - -(117) CometColumnarExchange -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#51, i_brand_id#126, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(118) CometColumnarToRow [codegen id : 242] -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] - -(119) HashAggregate [codegen id : 242] -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] -Keys [2]: [channel#51, i_brand_id#126] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#137, sum(number_sales#130)#138] -Results [6]: [channel#51, i_brand_id#126, null AS i_class_id#139, null AS i_category_id#140, sum(sum_sales#129)#137 AS sum(sum_sales)#141, sum(number_sales#130)#138 AS sum(number_sales)#142] - -(120) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] - -(121) CometColumnarToRow [codegen id : 322] -Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] - -(122) HashAggregate [codegen id : 322] -Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [3]: [channel#51, sum(sales#52)#106 AS sum_sales#146, sum(number_sales#53)#107 AS number_sales#147] - -(123) HashAggregate [codegen id : 322] -Input [3]: [channel#51, sum_sales#146, number_sales#147] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#146), partial_sum(number_sales#147)] -Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] -Results [4]: [channel#51, sum#151, isEmpty#152, sum#153] - -(124) CometColumnarExchange -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(125) CometColumnarToRow [codegen id : 323] -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] - -(126) HashAggregate [codegen id : 323] -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#146), sum(number_sales#147)] -Aggregate Attributes [2]: [sum(sum_sales#146)#154, sum(number_sales#147)#155] -Results [6]: [channel#51, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#146)#154 AS sum(sum_sales)#159, sum(number_sales#147)#155 AS sum(number_sales)#160] - -(127) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] - -(128) CometColumnarToRow [codegen id : 403] -Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] - -(129) HashAggregate [codegen id : 403] -Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [2]: [sum(sales#52)#106 AS sum_sales#164, sum(number_sales#53)#107 AS number_sales#165] - -(130) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#164, number_sales#165] +(80) CometFilter +Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Condition : isnotnull(ws_item_sk#71) + +(81) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#76] + +(82) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [1]: [ss_item_sk#76] +Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight + +(83) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] + +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight + +(85) CometProject +Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] + +(86) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#81] + +(87) CometBroadcastHashJoin +Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] +Right output [1]: [d_date_sk#81] +Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(88) CometProject +Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] +Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] + +(89) CometHashAggregate +Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] + +(90) CometExchange +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(91) CometHashAggregate +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] + +(92) CometFilter +Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(93) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] + +(94) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] + +(95) CometExchange +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(96) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(97) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] + +(98) CometHashAggregate +Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(99) CometHashAggregate +Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] + +(100) CometExchange +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(101) CometHashAggregate +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] + +(102) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] + +(103) CometHashAggregate +Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(104) CometHashAggregate +Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] + +(105) CometExchange +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(106) CometHashAggregate +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] + +(107) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] + +(108) CometHashAggregate +Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(109) CometHashAggregate +Input [3]: [channel#49, sum_sales#110, number_sales#111] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] + +(110) CometExchange +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(111) CometHashAggregate +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] + +(112) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] + +(113) CometHashAggregate +Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(114) CometHashAggregate +Input [2]: [sum_sales#118, number_sales#119] Keys: [] -Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [3]: [sum#169, isEmpty#170, sum#171] - -(131) CometColumnarExchange -Input [3]: [sum#169, isEmpty#170, sum#171] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] -(132) CometColumnarToRow [codegen id : 404] -Input [3]: [sum#169, isEmpty#170, sum#171] +(115) CometExchange +Input [3]: [sum#120, isEmpty#121, sum#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#169, isEmpty#170, sum#171] +(116) CometHashAggregate +Input [3]: [sum#120, isEmpty#121, sum#122] Keys: [] -Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] -Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] -Results [6]: [null AS channel#174, null AS i_brand_id#175, null AS i_class_id#176, null AS i_category_id#177, sum(sum_sales#164)#172 AS sum(sum_sales)#178, sum(number_sales#165)#173 AS sum(number_sales)#179] - -(134) Union - -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] + +(117) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] +Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] +Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] +Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] + +(118) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(136) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(119) CometExchange +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(137) CometHashAggregate -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(120) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -(138) CometTakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#108,number_sales#109]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109], 100, 0, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(121) CometTakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -(139) CometColumnarToRow [codegen id : 406] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(122) CometColumnarToRow [codegen id : 1] +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (159) -+- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- Union (155) - :- * Project (144) - : +- * BroadcastHashJoin Inner BuildRight (143) - : :- * CometColumnarToRow (141) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (142) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * CometColumnarToRow (146) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (145) - : +- ReusedExchange (147) - +- * Project (154) - +- * BroadcastHashJoin Inner BuildRight (153) - :- * CometColumnarToRow (151) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (150) - +- ReusedExchange (152) - - -(140) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* CometColumnarToRow (142) ++- CometHashAggregate (141) + +- CometExchange (140) + +- CometHashAggregate (139) + +- CometUnion (138) + :- CometProject (126) + : +- CometBroadcastHashJoin (125) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) + : +- ReusedExchange (124) + :- CometProject (133) + : +- CometBroadcastHashJoin (132) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) + : +- CometBroadcastExchange (131) + : +- CometProject (130) + : +- CometFilter (129) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) + +- ReusedExchange (135) + + +(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#182), dynamicpruningexpression(ss_sold_date_sk#182 IN dynamicpruning#183)] +PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] ReadSchema: struct -(141) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] +(124) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#147] -(142) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#184] +(125) CometBroadcastHashJoin +Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] +Right output [1]: [d_date_sk#147] +Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight -(143) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#182] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None +(126) CometProject +Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] +Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] -(144) Project [codegen id : 2] -Output [2]: [ss_quantity#180 AS quantity#185, ss_list_price#181 AS list_price#186] -Input [4]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182, d_date_sk#184] - -(145) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] +(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#189), dynamicpruningexpression(cs_sold_date_sk#189 IN dynamicpruning#190)] +PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] ReadSchema: struct -(146) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] +(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(129) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(147) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#191] +(130) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(148) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#189] -Right keys [1]: [d_date_sk#191] -Join type: Inner -Join condition: None +(131) CometBroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: [d_date_sk#154] -(149) Project [codegen id : 4] -Output [2]: [cs_quantity#187 AS quantity#192, cs_list_price#188 AS list_price#193] -Input [4]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189, d_date_sk#191] +(132) CometBroadcastHashJoin +Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] +Right output [1]: [d_date_sk#154] +Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight -(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] +(133) CometProject +Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] +Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] + +(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#196), dynamicpruningexpression(ws_sold_date_sk#196 IN dynamicpruning#197)] +PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] ReadSchema: struct -(151) CometColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] - -(152) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#198] +(135) ReusedExchange [Reuses operator id: 131] +Output [1]: [d_date_sk#162] -(153) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#196] -Right keys [1]: [d_date_sk#198] -Join type: Inner -Join condition: None +(136) CometBroadcastHashJoin +Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] +Right output [1]: [d_date_sk#162] +Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight -(154) Project [codegen id : 6] -Output [2]: [ws_quantity#194 AS quantity#199, ws_list_price#195 AS list_price#200] -Input [4]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196, d_date_sk#198] +(137) CometProject +Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] +Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] -(155) Union +(138) CometUnion +Child 0 Input [2]: [quantity#148, list_price#149] +Child 1 Input [2]: [quantity#156, list_price#157] +Child 2 Input [2]: [quantity#163, list_price#164] -(156) HashAggregate [codegen id : 7] -Input [2]: [quantity#185, list_price#186] +(139) CometHashAggregate +Input [2]: [quantity#148, list_price#149] Keys: [] -Functions [1]: [partial_avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] -Aggregate Attributes [2]: [sum#201, count#202] -Results [2]: [sum#203, count#204] +Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] -(157) CometColumnarExchange -Input [2]: [sum#203, count#204] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] +(140) CometExchange +Input [2]: [sum#165, count#166] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(158) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#203, count#204] - -(159) HashAggregate [codegen id : 8] -Input [2]: [sum#203, count#204] +(141) CometHashAggregate +Input [2]: [sum#165, count#166] Keys: [] -Functions [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] -Aggregate Attributes [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205] -Results [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205 AS average_sales#206] +Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] + +(142) CometColumnarToRow [codegen id : 1] +Input [1]: [average_sales#167] -Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#182 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#189 IN dynamicpruning#190 -BroadcastExchange (164) -+- * CometColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (160) +Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 +BroadcastExchange (147) ++- * CometColumnarToRow (146) + +- CometProject (145) + +- CometFilter (144) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) -(160) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#191, d_year#207] +(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#191, d_year#207] -Condition : (((isnotnull(d_year#207) AND (d_year#207 >= 1998)) AND (d_year#207 <= 2000)) AND isnotnull(d_date_sk#191)) +(144) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(162) CometProject -Input [2]: [d_date_sk#191, d_year#207] -Arguments: [d_date_sk#191], [d_date_sk#191] +(145) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(163) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#191] +(146) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#154] -(164) BroadcastExchange -Input [1]: [d_date_sk#191] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +(147) BroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#196 IN dynamicpruning#190 +Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * CometColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (165) +BroadcastExchange (152) ++- * CometColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) -(165) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#208, d_moy#209] +(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(166) CometFilter -Input [3]: [d_date_sk#42, d_year#208, d_moy#209] -Condition : ((((isnotnull(d_year#208) AND isnotnull(d_moy#209)) AND (d_year#208 = 2000)) AND (d_moy#209 = 11)) AND isnotnull(d_date_sk#42)) +(149) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(167) CometProject -Input [3]: [d_date_sk#42, d_year#208, d_moy#209] -Arguments: [d_date_sk#42], [d_date_sk#42] +(150) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] -(168) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(151) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(169) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(152) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * CometColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (170) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (157) ++- * CometColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) -(170) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#210] +(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#168] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(171) CometFilter -Input [2]: [d_date_sk#25, d_year#210] -Condition : (((isnotnull(d_year#210) AND (d_year#210 >= 1999)) AND (d_year#210 <= 2001)) AND isnotnull(d_date_sk#25)) +(154) CometFilter +Input [2]: [d_date_sk#26, d_year#168] +Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) -(172) CometProject -Input [2]: [d_date_sk#25, d_year#210] -Arguments: [d_date_sk#25], [d_date_sk#25] +(155) CometProject +Input [2]: [d_date_sk#26, d_year#168] +Arguments: [d_date_sk#26], [d_date_sk#26] -(173) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(156) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(174) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +(157) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index 13f50df6c9..83deb45432 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -1,2779 +1,2350 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 1067 out of 2302 eligible operators (46%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt index b3f0140800..758e917335 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt @@ -1,276 +1,179 @@ -WholeStageCodegen (406) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] + CometExchange #16 + CometHashAggregate [quantity,list_price] [sum,count] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + ReusedExchange [d_date_sk] #18 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #3 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #5 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [brand_id,class_id,category_id] #6 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometExchange [brand_id,class_id,category_id] #7 + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [d_date_sk] #12 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #5 + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #19 + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #20 + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel,i_brand_id,i_class_id] #21 + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel,i_brand_id] #22 + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] + CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel] #23 + CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange #24 + CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt index 7d029ef698..a727c5e743 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt @@ -1,143 +1,126 @@ == Physical Plan == -* CometColumnarToRow (139) -+- CometTakeOrderedAndProject (138) - +- CometHashAggregate (137) - +- CometColumnarExchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (105) - : +- * CometColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- * HashAggregate (102) - : +- Union (101) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) - : : +- CometColumnarExchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometColumnarExchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * CometColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * CometColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * CometColumnarToRow (39) - : : : : : : +- CometFilter (38) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (58) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (61) - : :- * Filter (84) - : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) - : : +- CometColumnarExchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * CometColumnarToRow (71) - : : : : : +- CometFilter (70) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometColumnarExchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : : :- * CometColumnarToRow (87) - : : : : +- CometFilter (86) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (85) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- CometColumnarExchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) - : +- ReusedExchange (106) - :- * HashAggregate (119) - : +- * CometColumnarToRow (118) - : +- CometColumnarExchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (126) - : +- * CometColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- * HashAggregate (123) - : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (133) - +- * CometColumnarToRow (132) - +- CometColumnarExchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- * CometColumnarToRow (128) - +- ReusedExchange (127) +* CometColumnarToRow (122) ++- CometTakeOrderedAndProject (121) + +- CometHashAggregate (120) + +- CometExchange (119) + +- CometHashAggregate (118) + +- CometUnion (117) + :- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometUnion (93) + : :- CometFilter (64) + : : +- CometHashAggregate (63) + : : +- CometExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometBroadcastHashJoin (47) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (46) + : : : : +- CometProject (45) + : : : : +- CometBroadcastHashJoin (44) + : : : : :- CometFilter (4) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : : +- CometBroadcastExchange (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometHashAggregate (32) + : : : : : +- CometExchange (31) + : : : : : +- CometHashAggregate (30) + : : : : : +- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (6) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometBroadcastHashJoin (23) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) + : : : : : : +- CometBroadcastExchange (22) + : : : : : : +- CometProject (21) + : : : : : : +- CometBroadcastHashJoin (20) + : : : : : : :- CometProject (15) + : : : : : : : +- CometBroadcastHashJoin (14) + : : : : : : : :- CometFilter (10) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) + : : : : : : : +- CometBroadcastExchange (13) + : : : : : : : +- CometFilter (12) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) + : : : : : : +- CometBroadcastExchange (19) + : : : : : : +- CometProject (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) + : : : : : +- ReusedExchange (27) + : : : : +- CometBroadcastExchange (41) + : : : : +- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometProject (37) + : : : : : +- CometBroadcastHashJoin (36) + : : : : : :- CometFilter (34) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) + : : : : : +- ReusedExchange (35) + : : : : +- ReusedExchange (38) + : : : +- CometBroadcastExchange (52) + : : : +- CometBroadcastHashJoin (51) + : : : :- CometFilter (49) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) + : : : +- ReusedExchange (50) + : : +- CometBroadcastExchange (58) + : : +- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + : :- CometFilter (78) + : : +- CometHashAggregate (77) + : : +- CometExchange (76) + : : +- CometHashAggregate (75) + : : +- CometProject (74) + : : +- CometBroadcastHashJoin (73) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometBroadcastHashJoin (68) + : : : : :- CometFilter (66) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) + : : : : +- ReusedExchange (67) + : : : +- ReusedExchange (69) + : : +- ReusedExchange (72) + : +- CometFilter (92) + : +- CometHashAggregate (91) + : +- CometExchange (90) + : +- CometHashAggregate (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometBroadcastHashJoin (82) + : : : :- CometFilter (80) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) + : : : +- ReusedExchange (81) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- CometHashAggregate (101) + : +- CometExchange (100) + : +- CometHashAggregate (99) + : +- CometHashAggregate (98) + : +- ReusedExchange (97) + :- CometHashAggregate (106) + : +- CometExchange (105) + : +- CometHashAggregate (104) + : +- CometHashAggregate (103) + : +- ReusedExchange (102) + :- CometHashAggregate (111) + : +- CometExchange (110) + : +- CometHashAggregate (109) + : +- CometHashAggregate (108) + : +- ReusedExchange (107) + +- CometHashAggregate (116) + +- CometExchange (115) + +- CometHashAggregate (114) + +- CometHashAggregate (113) + +- ReusedExchange (112) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -152,24 +135,18 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] @@ -177,28 +154,22 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(8) CometFilter +(6) CometFilter Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) CometColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(8) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -206,815 +177,738 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(10) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) CometColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(12) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) CometColumnarToRow [codegen id : 1] +(13) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) BroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(14) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(15) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] + +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) + +(18) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] -(22) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#24] +(19) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(21) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(22) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(23) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(24) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(26) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#25] +(27) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(29) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(30) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] -(34) CometColumnarExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(31) CometExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(35) CometHashAggregate -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) - -(39) CometColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] - -(43) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#36] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] - -(46) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(34) CometFilter +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) + +(35) ReusedExchange [Reuses operator id: 13] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] + +(36) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight + +(37) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] + +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#37] + +(39) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight + +(40) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] + +(41) CometBroadcastExchange +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] + +(42) CometBroadcastHashJoin +Left output [3]: [brand_id#27, class_id#28, category_id#29] +Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight + +(43) CometBroadcastExchange +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [brand_id#27, class_id#28, category_id#29] + +(44) CometBroadcastHashJoin +Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Right output [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight + +(45) CometProject +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] +Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] + +(46) CometBroadcastExchange +Input [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#38] + +(47) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [1]: [ss_item_sk#38] +Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight + +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(54) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +(49) CometFilter +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(50) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#38] -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#37] +(51) CometBroadcastHashJoin +Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [ss_item_sk#38] +Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] -Join type: LeftSemi -Join condition: None +(52) CometBroadcastExchange +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(58) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(53) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None +(54) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct -(61) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#42] +(56) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(57) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(58) CometBroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: [d_date_sk#43] -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(59) CometBroadcastHashJoin +Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Right output [1]: [d_date_sk#43] +Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight -(65) CometColumnarExchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(60) CometProject +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] +Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] + +(61) CometHashAggregate +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +(62) CometExchange +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) CometHashAggregate +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] -(68) Filter [codegen id : 26] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(64) CometFilter +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -Condition : isnotnull(cs_item_sk#56) - -(71) CometColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(75) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 51] -Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(77) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#66] - -(78) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 51] -Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(80) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(83) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] -Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] - -(84) Filter [codegen id : 52] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +(66) CometFilter +Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Condition : isnotnull(cs_item_sk#54) + +(67) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#59] + +(68) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [1]: [ss_item_sk#59] +Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight + +(69) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] + +(70) CometBroadcastHashJoin +Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] +Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight + +(71) CometProject +Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] +Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] + +(72) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#64] + +(73) CometBroadcastHashJoin +Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] +Right output [1]: [d_date_sk#64] +Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight + +(74) CometProject +Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] +Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] + +(75) CometHashAggregate +Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] + +(76) CometExchange +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(77) CometHashAggregate +Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] +Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] +Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] + +(78) CometFilter +Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) CometFilter -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_item_sk#78) - -(87) CometColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] - -(88) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#83] - -(89) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [ss_item_sk#83] -Join type: LeftSemi -Join condition: None - -(90) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [i_item_sk#84] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 77] -Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] - -(93) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#88] - -(94) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 77] -Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] - -(96) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] - -(97) CometColumnarExchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] - -(99) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] -Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] - -(100) Filter [codegen id : 78] -Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] -Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) - -(101) Union - -(102) HashAggregate [codegen id : 79] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] -Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] - -(103) CometColumnarExchange -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(104) CometColumnarToRow [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] - -(105) HashAggregate [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] - -(106) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] - -(107) CometColumnarToRow [codegen id : 160] -Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] - -(108) HashAggregate [codegen id : 160] -Input [7]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#110, i_class_id#111, i_category_id#112] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [5]: [channel#51, i_brand_id#110, i_class_id#111, sum(sales#52)#106 AS sum_sales#113, sum(number_sales#53)#107 AS number_sales#114] - -(109) HashAggregate [codegen id : 160] -Input [5]: [channel#51, i_brand_id#110, i_class_id#111, sum_sales#113, number_sales#114] -Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] -Functions [2]: [partial_sum(sum_sales#113), partial_sum(number_sales#114)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] - -(110) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#51, i_brand_id#110, i_class_id#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(111) CometColumnarToRow [codegen id : 161] -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] - -(112) HashAggregate [codegen id : 161] -Input [6]: [channel#51, i_brand_id#110, i_class_id#111, sum#118, isEmpty#119, sum#120] -Keys [3]: [channel#51, i_brand_id#110, i_class_id#111] -Functions [2]: [sum(sum_sales#113), sum(number_sales#114)] -Aggregate Attributes [2]: [sum(sum_sales#113)#121, sum(number_sales#114)#122] -Results [6]: [channel#51, i_brand_id#110, i_class_id#111, null AS i_category_id#123, sum(sum_sales#113)#121 AS sum(sum_sales)#124, sum(number_sales#114)#122 AS sum(number_sales)#125] - -(113) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] - -(114) CometColumnarToRow [codegen id : 241] -Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] - -(115) HashAggregate [codegen id : 241] -Input [7]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#126, i_class_id#127, i_category_id#128] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [4]: [channel#51, i_brand_id#126, sum(sales#52)#106 AS sum_sales#129, sum(number_sales#53)#107 AS number_sales#130] - -(116) HashAggregate [codegen id : 241] -Input [4]: [channel#51, i_brand_id#126, sum_sales#129, number_sales#130] -Keys [2]: [channel#51, i_brand_id#126] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] -Results [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] - -(117) CometColumnarExchange -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#51, i_brand_id#126, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(118) CometColumnarToRow [codegen id : 242] -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] - -(119) HashAggregate [codegen id : 242] -Input [5]: [channel#51, i_brand_id#126, sum#134, isEmpty#135, sum#136] -Keys [2]: [channel#51, i_brand_id#126] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#137, sum(number_sales#130)#138] -Results [6]: [channel#51, i_brand_id#126, null AS i_class_id#139, null AS i_category_id#140, sum(sum_sales#129)#137 AS sum(sum_sales)#141, sum(number_sales#130)#138 AS sum(number_sales)#142] - -(120) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] - -(121) CometColumnarToRow [codegen id : 322] -Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] - -(122) HashAggregate [codegen id : 322] -Input [7]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#143, i_class_id#144, i_category_id#145] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [3]: [channel#51, sum(sales#52)#106 AS sum_sales#146, sum(number_sales#53)#107 AS number_sales#147] - -(123) HashAggregate [codegen id : 322] -Input [3]: [channel#51, sum_sales#146, number_sales#147] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#146), partial_sum(number_sales#147)] -Aggregate Attributes [3]: [sum#148, isEmpty#149, sum#150] -Results [4]: [channel#51, sum#151, isEmpty#152, sum#153] - -(124) CometColumnarExchange -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(125) CometColumnarToRow [codegen id : 323] -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] - -(126) HashAggregate [codegen id : 323] -Input [4]: [channel#51, sum#151, isEmpty#152, sum#153] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#146), sum(number_sales#147)] -Aggregate Attributes [2]: [sum(sum_sales#146)#154, sum(number_sales#147)#155] -Results [6]: [channel#51, null AS i_brand_id#156, null AS i_class_id#157, null AS i_category_id#158, sum(sum_sales#146)#154 AS sum(sum_sales)#159, sum(number_sales#147)#155 AS sum(number_sales)#160] - -(127) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] - -(128) CometColumnarToRow [codegen id : 403] -Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] - -(129) HashAggregate [codegen id : 403] -Input [7]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#161, i_class_id#162, i_category_id#163] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [2]: [sum(sales#52)#106 AS sum_sales#164, sum(number_sales#53)#107 AS number_sales#165] - -(130) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#164, number_sales#165] +(80) CometFilter +Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Condition : isnotnull(ws_item_sk#71) + +(81) ReusedExchange [Reuses operator id: 46] +Output [1]: [ss_item_sk#76] + +(82) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [1]: [ss_item_sk#76] +Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight + +(83) ReusedExchange [Reuses operator id: 52] +Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] + +(84) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] +Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight + +(85) CometProject +Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] +Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] + +(86) ReusedExchange [Reuses operator id: 58] +Output [1]: [d_date_sk#81] + +(87) CometBroadcastHashJoin +Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] +Right output [1]: [d_date_sk#81] +Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(88) CometProject +Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] +Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] + +(89) CometHashAggregate +Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] + +(90) CometExchange +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(91) CometHashAggregate +Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] +Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] +Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] + +(92) CometFilter +Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] +Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) + +(93) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] +Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] + +(94) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] + +(95) CometExchange +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(96) CometHashAggregate +Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(97) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] + +(98) CometHashAggregate +Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(99) CometHashAggregate +Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] + +(100) CometExchange +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(101) CometHashAggregate +Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] +Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] +Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] + +(102) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] + +(103) CometHashAggregate +Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(104) CometHashAggregate +Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] + +(105) CometExchange +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(106) CometHashAggregate +Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] +Keys [2]: [channel#49, i_brand_id#99] +Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] + +(107) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] + +(108) CometHashAggregate +Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(109) CometHashAggregate +Input [3]: [channel#49, sum_sales#110, number_sales#111] +Keys [1]: [channel#49] +Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] + +(110) CometExchange +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(111) CometHashAggregate +Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] +Keys [1]: [channel#49] +Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] + +(112) ReusedExchange [Reuses operator id: 95] +Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] + +(113) CometHashAggregate +Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] +Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] +Functions [2]: [sum(sales#50), sum(number_sales#51)] + +(114) CometHashAggregate +Input [2]: [sum_sales#118, number_sales#119] Keys: [] -Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [3]: [sum#169, isEmpty#170, sum#171] - -(131) CometColumnarExchange -Input [3]: [sum#169, isEmpty#170, sum#171] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] +Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] -(132) CometColumnarToRow [codegen id : 404] -Input [3]: [sum#169, isEmpty#170, sum#171] +(115) CometExchange +Input [3]: [sum#120, isEmpty#121, sum#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#169, isEmpty#170, sum#171] +(116) CometHashAggregate +Input [3]: [sum#120, isEmpty#121, sum#122] Keys: [] -Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] -Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] -Results [6]: [null AS channel#174, null AS i_brand_id#175, null AS i_class_id#176, null AS i_category_id#177, sum(sum_sales#164)#172 AS sum(sum_sales)#178, sum(number_sales#165)#173 AS sum(number_sales)#179] - -(134) Union - -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] + +(117) CometUnion +Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] +Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] +Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] +Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] + +(118) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -(136) CometColumnarExchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(119) CometExchange +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(137) CometHashAggregate -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(120) CometHashAggregate +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] Functions: [] -(138) CometTakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#51 ASC NULLS FIRST,i_brand_id#39 ASC NULLS FIRST,i_class_id#40 ASC NULLS FIRST,i_category_id#41 ASC NULLS FIRST], output=[channel#51,i_brand_id#39,i_class_id#40,i_category_id#41,sum_sales#108,number_sales#109]), [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109], 100, 0, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(121) CometTakeOrderedAndProject +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -(139) CometColumnarToRow [codegen id : 406] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(122) CometColumnarToRow [codegen id : 1] +Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] ===== Subqueries ===== -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (159) -+- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- Union (155) - :- * Project (144) - : +- * BroadcastHashJoin Inner BuildRight (143) - : :- * CometColumnarToRow (141) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (142) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * CometColumnarToRow (146) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (145) - : +- ReusedExchange (147) - +- * Project (154) - +- * BroadcastHashJoin Inner BuildRight (153) - :- * CometColumnarToRow (151) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (150) - +- ReusedExchange (152) - - -(140) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] +Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +* CometColumnarToRow (142) ++- CometHashAggregate (141) + +- CometExchange (140) + +- CometHashAggregate (139) + +- CometUnion (138) + :- CometProject (126) + : +- CometBroadcastHashJoin (125) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) + : +- ReusedExchange (124) + :- CometProject (133) + : +- CometBroadcastHashJoin (132) + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) + : +- CometBroadcastExchange (131) + : +- CometProject (130) + : +- CometFilter (129) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) + +- CometProject (137) + +- CometBroadcastHashJoin (136) + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) + +- ReusedExchange (135) + + +(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#182), dynamicpruningexpression(ss_sold_date_sk#182 IN dynamicpruning#183)] +PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] ReadSchema: struct -(141) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182] +(124) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#147] -(142) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#184] +(125) CometBroadcastHashJoin +Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] +Right output [1]: [d_date_sk#147] +Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight -(143) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#182] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None +(126) CometProject +Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] +Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] -(144) Project [codegen id : 2] -Output [2]: [ss_quantity#180 AS quantity#185, ss_list_price#181 AS list_price#186] -Input [4]: [ss_quantity#180, ss_list_price#181, ss_sold_date_sk#182, d_date_sk#184] - -(145) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] +(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#189), dynamicpruningexpression(cs_sold_date_sk#189 IN dynamicpruning#190)] +PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] ReadSchema: struct -(146) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189] +(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(129) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(147) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#191] +(130) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(148) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#189] -Right keys [1]: [d_date_sk#191] -Join type: Inner -Join condition: None +(131) CometBroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: [d_date_sk#154] -(149) Project [codegen id : 4] -Output [2]: [cs_quantity#187 AS quantity#192, cs_list_price#188 AS list_price#193] -Input [4]: [cs_quantity#187, cs_list_price#188, cs_sold_date_sk#189, d_date_sk#191] +(132) CometBroadcastHashJoin +Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] +Right output [1]: [d_date_sk#154] +Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight -(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] +(133) CometProject +Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] +Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] + +(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#196), dynamicpruningexpression(ws_sold_date_sk#196 IN dynamicpruning#197)] +PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] ReadSchema: struct -(151) CometColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196] - -(152) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#198] +(135) ReusedExchange [Reuses operator id: 131] +Output [1]: [d_date_sk#162] -(153) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#196] -Right keys [1]: [d_date_sk#198] -Join type: Inner -Join condition: None +(136) CometBroadcastHashJoin +Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] +Right output [1]: [d_date_sk#162] +Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight -(154) Project [codegen id : 6] -Output [2]: [ws_quantity#194 AS quantity#199, ws_list_price#195 AS list_price#200] -Input [4]: [ws_quantity#194, ws_list_price#195, ws_sold_date_sk#196, d_date_sk#198] +(137) CometProject +Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] +Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] -(155) Union +(138) CometUnion +Child 0 Input [2]: [quantity#148, list_price#149] +Child 1 Input [2]: [quantity#156, list_price#157] +Child 2 Input [2]: [quantity#163, list_price#164] -(156) HashAggregate [codegen id : 7] -Input [2]: [quantity#185, list_price#186] +(139) CometHashAggregate +Input [2]: [quantity#148, list_price#149] Keys: [] -Functions [1]: [partial_avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] -Aggregate Attributes [2]: [sum#201, count#202] -Results [2]: [sum#203, count#204] +Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] -(157) CometColumnarExchange -Input [2]: [sum#203, count#204] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] +(140) CometExchange +Input [2]: [sum#165, count#166] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(158) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#203, count#204] - -(159) HashAggregate [codegen id : 8] -Input [2]: [sum#203, count#204] +(141) CometHashAggregate +Input [2]: [sum#165, count#166] Keys: [] -Functions [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))] -Aggregate Attributes [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205] -Results [1]: [avg((cast(quantity#185 as decimal(10,0)) * list_price#186))#205 AS average_sales#206] +Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] + +(142) CometColumnarToRow [codegen id : 1] +Input [1]: [average_sales#167] -Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#182 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#189 IN dynamicpruning#190 -BroadcastExchange (164) -+- * CometColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (160) +Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 +BroadcastExchange (147) ++- * CometColumnarToRow (146) + +- CometProject (145) + +- CometFilter (144) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) -(160) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#191, d_year#207] +(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#154, d_year#155] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(161) CometFilter -Input [2]: [d_date_sk#191, d_year#207] -Condition : (((isnotnull(d_year#207) AND (d_year#207 >= 1998)) AND (d_year#207 <= 2000)) AND isnotnull(d_date_sk#191)) +(144) CometFilter +Input [2]: [d_date_sk#154, d_year#155] +Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) -(162) CometProject -Input [2]: [d_date_sk#191, d_year#207] -Arguments: [d_date_sk#191], [d_date_sk#191] +(145) CometProject +Input [2]: [d_date_sk#154, d_year#155] +Arguments: [d_date_sk#154], [d_date_sk#154] -(163) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#191] +(146) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#154] -(164) BroadcastExchange -Input [1]: [d_date_sk#191] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +(147) BroadcastExchange +Input [1]: [d_date_sk#154] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#196 IN dynamicpruning#190 +Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * CometColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (165) +BroadcastExchange (152) ++- * CometColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) -(165) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#208, d_moy#209] +(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(166) CometFilter -Input [3]: [d_date_sk#42, d_year#208, d_moy#209] -Condition : ((((isnotnull(d_year#208) AND isnotnull(d_moy#209)) AND (d_year#208 = 2000)) AND (d_moy#209 = 11)) AND isnotnull(d_date_sk#42)) +(149) CometFilter +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) -(167) CometProject -Input [3]: [d_date_sk#42, d_year#208, d_moy#209] -Arguments: [d_date_sk#42], [d_date_sk#42] +(150) CometProject +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Arguments: [d_date_sk#43], [d_date_sk#43] -(168) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(151) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(169) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(152) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * CometColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (170) +Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (157) ++- * CometColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) -(170) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#210] +(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#168] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(171) CometFilter -Input [2]: [d_date_sk#25, d_year#210] -Condition : (((isnotnull(d_year#210) AND (d_year#210 >= 1999)) AND (d_year#210 <= 2001)) AND isnotnull(d_date_sk#25)) +(154) CometFilter +Input [2]: [d_date_sk#26, d_year#168] +Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) -(172) CometProject -Input [2]: [d_date_sk#25, d_year#210] -Arguments: [d_date_sk#25], [d_date_sk#25] +(155) CometProject +Input [2]: [d_date_sk#26, d_year#168] +Arguments: [d_date_sk#26], [d_date_sk#26] -(173) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(156) CometColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(174) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +(157) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt index 13f50df6c9..83deb45432 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt @@ -1,2779 +1,2350 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 1067 out of 2302 eligible operators (46%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt index b3f0140800..758e917335 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt @@ -1,276 +1,179 @@ -WholeStageCodegen (406) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] + CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + Subquery #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] + CometExchange #16 + CometHashAggregate [quantity,list_price] [sum,count] + CometUnion [quantity,list_price] + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #18 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + ReusedExchange [d_date_sk] #18 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #3 + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] + CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk] #5 + CometProject [i_item_sk] [ss_item_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [brand_id,class_id,category_id] #6 + CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + CometHashAggregate [brand_id,class_id,category_id] + CometExchange [brand_id,class_id,category_id] #7 + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange [d_date_sk] #12 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometFilter [ws_item_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + ReusedExchange [ss_item_sk] #5 + CometBroadcastExchange [d_date_sk] #15 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #19 + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] + CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] + ReusedSubquery [average_sales] #3 + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometExchange [i_brand_id,i_class_id,i_category_id] #20 + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] + CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] + CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 + ReusedExchange [d_date_sk] #15 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel,i_brand_id,i_class_id] #21 + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel,i_brand_id] #22 + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] + CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange [channel] #23 + CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] + CometExchange #24 + CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt index cb77e85abc..72e9bd1b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt @@ -1,167 +1,150 @@ == Physical Plan == -* CometColumnarToRow (163) -+- CometTakeOrderedAndProject (162) - +- RowToColumnar (161) - +- Union (160) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * Project (56) - : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * Project (50) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : :- * CometColumnarToRow (47) - : : : : : : : +- CometFilter (46) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (45) - : : : : : : +- ReusedExchange (48) - : : : : : +- ReusedExchange (51) - : : : : +- ReusedExchange (54) - : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) - : : : +- CometProject (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (57) - : : +- ReusedExchange (64) - : +- ReusedExchange (67) - :- * HashAggregate (102) - : +- * CometColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (85) - : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : :- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (74) - : : : : : : +- ReusedExchange (77) - : : : : : +- ReusedExchange (80) - : : : : +- ReusedExchange (83) - : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) - : : : +- CometProject (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (86) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (131) - : +- * CometColumnarToRow (130) - : +- CometColumnarExchange (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * CometColumnarToRow (105) - : : : : : : : +- CometFilter (104) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (159) - +- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- * Project (155) - +- * BroadcastHashJoin Inner BuildRight (154) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * Project (146) - : : +- * BroadcastHashJoin Inner BuildRight (145) - : : :- * Project (143) - : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : :- * Project (140) - : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : :- * Project (137) - : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : :- * CometColumnarToRow (134) - : : : : : : +- CometFilter (133) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (132) - : : : : : +- ReusedExchange (135) - : : : : +- ReusedExchange (138) - : : : +- ReusedExchange (141) - : : +- ReusedExchange (144) - : +- ReusedExchange (147) - +- BroadcastExchange (153) - +- * CometColumnarToRow (152) - +- CometFilter (151) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (150) +* CometColumnarToRow (146) ++- CometTakeOrderedAndProject (145) + +- CometUnion (144) + :- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (19) + : : : : +- CometBroadcastHashJoin (18) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) + : : : : +- CometBroadcastExchange (17) + : : : : +- CometFilter (16) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) + : : : +- CometBroadcastExchange (23) + : : : +- CometProject (22) + : : : +- CometFilter (21) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) + : : +- CometBroadcastExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) + :- CometHashAggregate (66) + : +- CometExchange (65) + : +- CometHashAggregate (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometProject (51) + : : : : +- CometBroadcastHashJoin (50) + : : : : :- CometProject (48) + : : : : : +- CometBroadcastHashJoin (47) + : : : : : :- CometProject (45) + : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : :- CometFilter (42) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- ReusedExchange (43) + : : : : : +- ReusedExchange (46) + : : : : +- ReusedExchange (49) + : : : +- CometBroadcastExchange (55) + : : : +- CometProject (54) + : : : +- CometFilter (53) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) + : : +- ReusedExchange (58) + : +- ReusedExchange (61) + :- CometHashAggregate (92) + : +- CometExchange (91) + : +- CometHashAggregate (90) + : +- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometProject (86) + : : +- CometBroadcastHashJoin (85) + : : :- CometProject (83) + : : : +- CometBroadcastHashJoin (82) + : : : :- CometProject (77) + : : : : +- CometBroadcastHashJoin (76) + : : : : :- CometProject (74) + : : : : : +- CometBroadcastHashJoin (73) + : : : : : :- CometProject (71) + : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : :- CometFilter (68) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) + : : : : : : +- ReusedExchange (69) + : : : : : +- ReusedExchange (72) + : : : : +- ReusedExchange (75) + : : : +- CometBroadcastExchange (81) + : : : +- CometProject (80) + : : : +- CometFilter (79) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) + : : +- ReusedExchange (84) + : +- ReusedExchange (87) + :- CometHashAggregate (118) + : +- CometExchange (117) + : +- CometHashAggregate (116) + : +- CometProject (115) + : +- CometBroadcastHashJoin (114) + : :- CometProject (112) + : : +- CometBroadcastHashJoin (111) + : : :- CometProject (109) + : : : +- CometBroadcastHashJoin (108) + : : : :- CometProject (103) + : : : : +- CometBroadcastHashJoin (102) + : : : : :- CometProject (100) + : : : : : +- CometBroadcastHashJoin (99) + : : : : : :- CometProject (97) + : : : : : : +- CometBroadcastHashJoin (96) + : : : : : : :- CometFilter (94) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (98) + : : : : +- ReusedExchange (101) + : : : +- CometBroadcastExchange (107) + : : : +- CometProject (106) + : : : +- CometFilter (105) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) + : : +- ReusedExchange (110) + : +- ReusedExchange (113) + +- CometHashAggregate (143) + +- CometExchange (142) + +- CometHashAggregate (141) + +- CometProject (140) + +- CometBroadcastHashJoin (139) + :- CometProject (135) + : +- CometBroadcastHashJoin (134) + : :- CometProject (132) + : : +- CometBroadcastHashJoin (131) + : : :- CometProject (129) + : : : +- CometBroadcastHashJoin (128) + : : : :- CometProject (126) + : : : : +- CometBroadcastHashJoin (125) + : : : : :- CometProject (123) + : : : : : +- CometBroadcastHashJoin (122) + : : : : : :- CometFilter (120) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) + : : : : : +- ReusedExchange (121) + : : : : +- ReusedExchange (124) + : : : +- ReusedExchange (127) + : : +- ReusedExchange (130) + : +- ReusedExchange (133) + +- CometBroadcastExchange (138) + +- CometFilter (137) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -176,777 +159,688 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [cd_demo_sk#20] -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(26) CometProject +(22) CometProject Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange +(23) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +(25) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(31) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#26] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#26, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) + +(28) CometProject +Input [2]: [d_date_sk#26, d_year#27] +Arguments: [d_date_sk#26], [d_date_sk#26] + +(29) CometBroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Right output [1]: [d_date_sk#26] +Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +(31) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#28, i_item_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(42) CometColumnarExchange -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(44) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +(33) CometFilter +Input [2]: [i_item_sk#28, i_item_id#29] +Condition : isnotnull(i_item_sk#28) + +(34) CometProject +Input [2]: [i_item_sk#28, i_item_id#29] +Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] + +(35) CometBroadcastExchange +Input [2]: [i_item_sk#28, i_item_id#30] +Arguments: [i_item_sk#28, i_item_id#30] + +(36) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Right output [2]: [i_item_sk#28, i_item_id#30] +Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight + +(37) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] +Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] + +(38) CometHashAggregate +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] +Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] + +(39) CometExchange +Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(40) CometHashAggregate +Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] +Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] + +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(46) CometFilter -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) +(42) CometFilter +Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) -(47) CometColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +(43) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#66, cd_dep_count#67] -(48) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#93, cd_dep_count#94] +(44) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +Right output [2]: [cd_demo_sk#66, cd_dep_count#67] +Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#93] -Join type: Inner -Join condition: None +(45) CometProject +Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] +Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -(50) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#93, cd_dep_count#94] +(46) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -(51) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +(47) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] +Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#95] -Join type: Inner -Join condition: None +(48) CometProject +Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -(53) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +(49) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#72] -(54) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#99] +(50) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Right output [1]: [cd_demo_sk#72] +Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight -(55) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#96] -Right keys [1]: [cd_demo_sk#99] -Join type: Inner -Join condition: None +(51) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -(56) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98, cd_demo_sk#99] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(58) CometFilter -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) - -(59) CometProject -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Arguments: [ca_address_sk#100, ca_state#25, ca_country#102], [ca_address_sk#100, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) AS ca_state#25, ca_country#102] - -(60) CometColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] - -(61) BroadcastExchange -Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#97] -Right keys [1]: [ca_address_sk#100] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98, ca_address_sk#100, ca_state#25, ca_country#102] - -(64) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#103] - -(65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, d_date_sk#103] - -(67) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#104, i_item_id#29] - -(68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#104] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [10]: [i_item_id#29, ca_country#102, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#105, cast(cs_list_price#87 as decimal(12,2)) AS agg2#106, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#107, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#108, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#109, cast(c_birth_year#98 as decimal(12,2)) AS agg6#110, cast(cd_dep_count#94 as decimal(12,2)) AS agg7#111] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, i_item_sk#104, i_item_id#29] - -(70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#29, ca_country#102, ca_state#25, agg1#105, agg2#106, agg3#107, agg4#108, agg5#109, agg6#110, agg7#111] -Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] -Functions [7]: [partial_avg(agg1#105), partial_avg(agg2#106), partial_avg(agg3#107), partial_avg(agg4#108), partial_avg(agg5#109), partial_avg(agg6#110), partial_avg(agg7#111)] -Aggregate Attributes [14]: [sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] -Results [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] - -(71) CometColumnarExchange -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] -Arguments: hashpartitioning(i_item_id#29, ca_country#102, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometColumnarToRow [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] - -(73) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] -Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] -Functions [7]: [avg(agg1#105), avg(agg2#106), avg(agg3#107), avg(agg4#108), avg(agg5#109), avg(agg6#110), avg(agg7#111)] -Aggregate Attributes [7]: [avg(agg1#105)#140, avg(agg2#106)#141, avg(agg3#107)#142, avg(agg4#108)#143, avg(agg5#109)#144, avg(agg6#110)#145, avg(agg7#111)#146] -Results [11]: [i_item_id#29, ca_country#102, ca_state#25, null AS county#147, avg(agg1#105)#140 AS agg1#148, avg(agg2#106)#141 AS agg2#149, avg(agg3#107)#142 AS agg3#150, avg(agg4#108)#143 AS agg4#151, avg(agg5#109)#144 AS agg5#152, avg(agg6#110)#145 AS agg6#153, avg(agg7#111)#146 AS agg7#154] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +(53) CometFilter +Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) + +(54) CometProject +Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] + +(55) CometBroadcastExchange +Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] + +(56) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] +Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight + +(57) CometProject +Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] + +(58) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#76] + +(59) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] +Right output [1]: [d_date_sk#76] +Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight + +(60) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] + +(61) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#77, i_item_id#30] + +(62) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] +Right output [2]: [i_item_sk#77, i_item_id#30] +Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight + +(63) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] +Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] + +(64) CometHashAggregate +Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] +Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] + +(65) CometExchange +Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(66) CometHashAggregate +Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] +Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] + +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#163), dynamicpruningexpression(cs_sold_date_sk#163 IN dynamicpruning#164)] +PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(75) CometFilter -Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] -Condition : ((isnotnull(cs_bill_cdemo_sk#156) AND isnotnull(cs_bill_customer_sk#155)) AND isnotnull(cs_item_sk#157)) - -(76) CometColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +(68) CometFilter +Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) -(77) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#165, cd_dep_count#166] +(69) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#109, cd_dep_count#110] -(78) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#156] -Right keys [1]: [cd_demo_sk#165] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +Right output [2]: [cd_demo_sk#109, cd_dep_count#110] +Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight -(79) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166] -Input [11]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_demo_sk#165, cd_dep_count#166] +(71) CometProject +Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] +Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -(80) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +(72) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#155] -Right keys [1]: [c_customer_sk#167] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] +Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight -(82) Project [codegen id : 23] -Output [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -Input [13]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +(74) CometProject +Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -(83) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#171] +(75) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#115] -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#168] -Right keys [1]: [cd_demo_sk#171] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Right output [1]: [cd_demo_sk#115] +Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight -(85) Project [codegen id : 23] -Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] -Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] +(77) CometProject +Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(87) CometFilter -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#173, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#172)) - -(88) CometProject -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] - -(89) CometColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#172, ca_country#174] - -(90) BroadcastExchange -Input [2]: [ca_address_sk#172, ca_country#174] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(91) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#169] -Right keys [1]: [ca_address_sk#172] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 23] -Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] - -(93) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#175] - -(94) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#163] -Right keys [1]: [d_date_sk#175] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 23] -Output [9]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] - -(96) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#176, i_item_id#29] - -(97) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#157] -Right keys [1]: [i_item_sk#176] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 23] -Output [9]: [i_item_id#29, ca_country#174, cast(cs_quantity#158 as decimal(12,2)) AS agg1#177, cast(cs_list_price#159 as decimal(12,2)) AS agg2#178, cast(cs_coupon_amt#161 as decimal(12,2)) AS agg3#179, cast(cs_sales_price#160 as decimal(12,2)) AS agg4#180, cast(cs_net_profit#162 as decimal(12,2)) AS agg5#181, cast(c_birth_year#170 as decimal(12,2)) AS agg6#182, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#183] -Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#29] - -(99) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#29, ca_country#174, agg1#177, agg2#178, agg3#179, agg4#180, agg5#181, agg6#182, agg7#183] -Keys [2]: [i_item_id#29, ca_country#174] -Functions [7]: [partial_avg(agg1#177), partial_avg(agg2#178), partial_avg(agg3#179), partial_avg(agg4#180), partial_avg(agg5#181), partial_avg(agg6#182), partial_avg(agg7#183)] -Aggregate Attributes [14]: [sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197] -Results [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] - -(100) CometColumnarExchange -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] -Arguments: hashpartitioning(i_item_id#29, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(101) CometColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] - -(102) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] -Keys [2]: [i_item_id#29, ca_country#174] -Functions [7]: [avg(agg1#177), avg(agg2#178), avg(agg3#179), avg(agg4#180), avg(agg5#181), avg(agg6#182), avg(agg7#183)] -Aggregate Attributes [7]: [avg(agg1#177)#212, avg(agg2#178)#213, avg(agg3#179)#214, avg(agg4#180)#215, avg(agg5#181)#216, avg(agg6#182)#217, avg(agg7#183)#218] -Results [11]: [i_item_id#29, ca_country#174, null AS ca_state#219, null AS county#220, avg(agg1#177)#212 AS agg1#221, avg(agg2#178)#213 AS agg2#222, avg(agg3#179)#214 AS agg3#223, avg(agg4#180)#215 AS agg4#224, avg(agg5#181)#216 AS agg5#225, avg(agg6#182)#217 AS agg6#226, avg(agg7#183)#218 AS agg7#227] - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +(79) CometFilter +Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) + +(80) CometProject +Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] + +(81) CometBroadcastExchange +Input [2]: [ca_address_sk#116, ca_country#118] +Arguments: [ca_address_sk#116, ca_country#118] + +(82) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] +Right output [2]: [ca_address_sk#116, ca_country#118] +Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight + +(83) CometProject +Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] + +(84) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#119] + +(85) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] +Right output [1]: [d_date_sk#119] +Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight + +(86) CometProject +Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] + +(87) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#120, i_item_id#30] + +(88) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] +Right output [2]: [i_item_sk#120, i_item_id#30] +Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight + +(89) CometProject +Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] +Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] + +(90) CometHashAggregate +Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] +Keys [2]: [i_item_id#30, ca_country#118] +Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] + +(91) CometExchange +Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] +Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(92) CometHashAggregate +Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] +Keys [2]: [i_item_id#30, ca_country#118] +Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] + +(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#236), dynamicpruningexpression(cs_sold_date_sk#236 IN dynamicpruning#237)] +PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) CometFilter -Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] -Condition : ((isnotnull(cs_bill_cdemo_sk#229) AND isnotnull(cs_bill_customer_sk#228)) AND isnotnull(cs_item_sk#230)) +(94) CometFilter +Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) -(105) CometColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +(95) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#152, cd_dep_count#153] -(106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#238, cd_dep_count#239] +(96) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +Right output [2]: [cd_demo_sk#152, cd_dep_count#153] +Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#229] -Right keys [1]: [cd_demo_sk#238] -Join type: Inner -Join condition: None +(97) CometProject +Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] +Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -(108) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239] -Input [11]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_demo_sk#238, cd_dep_count#239] +(98) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -(109) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +(99) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] +Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight -(110) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#228] -Right keys [1]: [c_customer_sk#240] -Join type: Inner -Join condition: None +(100) CometProject +Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -(111) Project [codegen id : 31] -Output [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -Input [13]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +(101) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#158] -(112) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#244] +(102) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Right output [1]: [cd_demo_sk#158] +Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight -(113) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#241] -Right keys [1]: [cd_demo_sk#244] -Join type: Inner -Join condition: None +(103) CometProject +Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -(114) Project [codegen id : 31] -Output [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] -Input [12]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] - -(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#245, ca_state#246] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#159, ca_state#160] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(116) CometFilter -Input [2]: [ca_address_sk#245, ca_state#246] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#246, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#245)) - -(117) CometProject -Input [2]: [ca_address_sk#245, ca_state#246] -Arguments: [ca_address_sk#245], [ca_address_sk#245] - -(118) CometColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#245] - -(119) BroadcastExchange -Input [1]: [ca_address_sk#245] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#242] -Right keys [1]: [ca_address_sk#245] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [9]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243] -Input [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] - -(122) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#247] - -(123) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#236] -Right keys [1]: [d_date_sk#247] -Join type: Inner -Join condition: None - -(124) Project [codegen id : 31] -Output [8]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243] -Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243, d_date_sk#247] - -(125) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#248, i_item_id#29] - -(126) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#230] -Right keys [1]: [i_item_sk#248] -Join type: Inner -Join condition: None - -(127) Project [codegen id : 31] -Output [8]: [i_item_id#29, cast(cs_quantity#231 as decimal(12,2)) AS agg1#249, cast(cs_list_price#232 as decimal(12,2)) AS agg2#250, cast(cs_coupon_amt#234 as decimal(12,2)) AS agg3#251, cast(cs_sales_price#233 as decimal(12,2)) AS agg4#252, cast(cs_net_profit#235 as decimal(12,2)) AS agg5#253, cast(c_birth_year#243 as decimal(12,2)) AS agg6#254, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#255] -Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#29] - -(128) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#29, agg1#249, agg2#250, agg3#251, agg4#252, agg5#253, agg6#254, agg7#255] -Keys [1]: [i_item_id#29] -Functions [7]: [partial_avg(agg1#249), partial_avg(agg2#250), partial_avg(agg3#251), partial_avg(agg4#252), partial_avg(agg5#253), partial_avg(agg6#254), partial_avg(agg7#255)] -Aggregate Attributes [14]: [sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Results [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] - -(129) CometColumnarExchange -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(130) CometColumnarToRow [codegen id : 32] -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] - -(131) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] -Keys [1]: [i_item_id#29] -Functions [7]: [avg(agg1#249), avg(agg2#250), avg(agg3#251), avg(agg4#252), avg(agg5#253), avg(agg6#254), avg(agg7#255)] -Aggregate Attributes [7]: [avg(agg1#249)#284, avg(agg2#250)#285, avg(agg3#251)#286, avg(agg4#252)#287, avg(agg5#253)#288, avg(agg6#254)#289, avg(agg7#255)#290] -Results [11]: [i_item_id#29, null AS ca_country#291, null AS ca_state#292, null AS county#293, avg(agg1#249)#284 AS agg1#294, avg(agg2#250)#285 AS agg2#295, avg(agg3#251)#286 AS agg3#296, avg(agg4#252)#287 AS agg4#297, avg(agg5#253)#288 AS agg5#298, avg(agg6#254)#289 AS agg6#299, avg(agg7#255)#290 AS agg7#300] - -(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +(105) CometFilter +Input [2]: [ca_address_sk#159, ca_state#160] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) + +(106) CometProject +Input [2]: [ca_address_sk#159, ca_state#160] +Arguments: [ca_address_sk#159], [ca_address_sk#159] + +(107) CometBroadcastExchange +Input [1]: [ca_address_sk#159] +Arguments: [ca_address_sk#159] + +(108) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] +Right output [1]: [ca_address_sk#159] +Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight + +(109) CometProject +Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] + +(110) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#161] + +(111) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] +Right output [1]: [d_date_sk#161] +Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight + +(112) CometProject +Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] + +(113) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#162, i_item_id#30] + +(114) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] +Right output [2]: [i_item_sk#162, i_item_id#30] +Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight + +(115) CometProject +Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] +Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] + +(116) CometHashAggregate +Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] +Keys [1]: [i_item_id#30] +Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] + +(117) CometExchange +Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(118) CometHashAggregate +Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] +Keys [1]: [i_item_id#30] +Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] + +(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#309), dynamicpruningexpression(cs_sold_date_sk#309 IN dynamicpruning#310)] +PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(133) CometFilter -Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] -Condition : ((isnotnull(cs_bill_cdemo_sk#302) AND isnotnull(cs_bill_customer_sk#301)) AND isnotnull(cs_item_sk#303)) +(120) CometFilter +Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) -(134) CometColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +(121) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#194, cd_dep_count#195] -(135) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#311, cd_dep_count#312] +(122) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +Right output [2]: [cd_demo_sk#194, cd_dep_count#195] +Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight -(136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#302] -Right keys [1]: [cd_demo_sk#311] -Join type: Inner -Join condition: None +(123) CometProject +Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] +Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -(137) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312] -Input [11]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_demo_sk#311, cd_dep_count#312] +(124) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -(138) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +(125) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] +Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight -(139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#301] -Right keys [1]: [c_customer_sk#313] -Join type: Inner -Join condition: None +(126) CometProject +Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -(140) Project [codegen id : 39] -Output [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -Input [13]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +(127) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#200] -(141) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#317] +(128) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Right output [1]: [cd_demo_sk#200] +Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight -(142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#314] -Right keys [1]: [cd_demo_sk#317] -Join type: Inner -Join condition: None +(129) CometProject +Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -(143) Project [codegen id : 39] -Output [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] -Input [12]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] +(130) ReusedExchange [Reuses operator id: 107] +Output [1]: [ca_address_sk#201] -(144) ReusedExchange [Reuses operator id: 119] -Output [1]: [ca_address_sk#318] +(131) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] +Right output [1]: [ca_address_sk#201] +Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight -(145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#315] -Right keys [1]: [ca_address_sk#318] -Join type: Inner -Join condition: None +(132) CometProject +Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -(146) Project [codegen id : 39] -Output [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316] -Input [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] +(133) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#202] -(147) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#319] +(134) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] +Right output [1]: [d_date_sk#202] +Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight -(148) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#309] -Right keys [1]: [d_date_sk#319] -Join type: Inner -Join condition: None +(135) CometProject +Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -(149) Project [codegen id : 39] -Output [8]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316] -Input [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316, d_date_sk#319] - -(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#320] +(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#203] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(151) CometFilter -Input [1]: [i_item_sk#320] -Condition : isnotnull(i_item_sk#320) - -(152) CometColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#320] +(137) CometFilter +Input [1]: [i_item_sk#203] +Condition : isnotnull(i_item_sk#203) -(153) BroadcastExchange -Input [1]: [i_item_sk#320] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(138) CometBroadcastExchange +Input [1]: [i_item_sk#203] +Arguments: [i_item_sk#203] -(154) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#303] -Right keys [1]: [i_item_sk#320] -Join type: Inner -Join condition: None +(139) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] +Right output [1]: [i_item_sk#203] +Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight -(155) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#304 as decimal(12,2)) AS agg1#321, cast(cs_list_price#305 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#307 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#306 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#308 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] -Input [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316, i_item_sk#320] +(140) CometProject +Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] +Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] -(156) HashAggregate [codegen id : 39] -Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] +(141) CometHashAggregate +Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] Keys: [] -Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] -Aggregate Attributes [14]: [sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339, sum#340, count#341] -Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] -(157) CometColumnarExchange -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(142) CometExchange +Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(158) CometColumnarToRow [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] - -(159) HashAggregate [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +(143) CometHashAggregate +Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] Keys: [] -Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] -Aggregate Attributes [7]: [avg(agg1#321)#356, avg(agg2#322)#357, avg(agg3#323)#358, avg(agg4#324)#359, avg(agg5#325)#360, avg(agg6#326)#361, avg(agg7#327)#362] -Results [11]: [null AS i_item_id#363, null AS ca_country#364, null AS ca_state#365, null AS county#366, avg(agg1#321)#356 AS agg1#367, avg(agg2#322)#357 AS agg2#368, avg(agg3#323)#358 AS agg3#369, avg(agg4#324)#359 AS agg4#370, avg(agg5#325)#360 AS agg5#371, avg(agg6#326)#361 AS agg6#372, avg(agg7#327)#362 AS agg7#373] - -(160) Union +Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] -(161) RowToColumnar -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(144) CometUnion +Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] +Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] +Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] +Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] -(162) CometTakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#31 ASC NULLS FIRST,ca_state#32 ASC NULLS FIRST,ca_county#33 ASC NULLS FIRST,i_item_id#30 ASC NULLS FIRST], output=[i_item_id#30,ca_country#31,ca_state#32,ca_county#33,agg1#76,agg2#77,agg3#78,agg4#79,agg5#80,agg6#81,agg7#82]), [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82], 100, 0, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(145) CometTakeOrderedAndProject +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -(163) CometColumnarToRow [codegen id : 41] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(146) CometColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (168) -+- * CometColumnarToRow (167) - +- CometProject (166) - +- CometFilter (165) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (164) +BroadcastExchange (151) ++- * CometColumnarToRow (150) + +- CometProject (149) + +- CometFilter (148) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) -(164) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#374] +(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(165) CometFilter -Input [2]: [d_date_sk#26, d_year#374] -Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_date_sk#26)) +(148) CometFilter +Input [2]: [d_date_sk#26, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) -(166) CometProject -Input [2]: [d_date_sk#26, d_year#374] +(149) CometProject +Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(167) CometColumnarToRow [codegen id : 1] +(150) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(168) BroadcastExchange +(151) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#163 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#236 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#309 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index 7a8f23c282..b18a444bb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -1,255 +1,214 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 103 out of 210 eligible operators (49%). Final plan contains 43 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt index 91800fcca0..1572a2a240 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt @@ -1,245 +1,160 @@ -WholeStageCodegen (41) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - RowToColumnar - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country,ca_state] #9 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country] #11 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk,ca_country] #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id] #13 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk] #14 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange #15 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + ReusedExchange [ca_address_sk] #14 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [i_item_sk] #16 + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt index cb77e85abc..72e9bd1b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt @@ -1,167 +1,150 @@ == Physical Plan == -* CometColumnarToRow (163) -+- CometTakeOrderedAndProject (162) - +- RowToColumnar (161) - +- Union (160) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * Project (56) - : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * Project (50) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : :- * CometColumnarToRow (47) - : : : : : : : +- CometFilter (46) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (45) - : : : : : : +- ReusedExchange (48) - : : : : : +- ReusedExchange (51) - : : : : +- ReusedExchange (54) - : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) - : : : +- CometProject (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (57) - : : +- ReusedExchange (64) - : +- ReusedExchange (67) - :- * HashAggregate (102) - : +- * CometColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (85) - : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : :- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (74) - : : : : : : +- ReusedExchange (77) - : : : : : +- ReusedExchange (80) - : : : : +- ReusedExchange (83) - : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) - : : : +- CometProject (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (86) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (131) - : +- * CometColumnarToRow (130) - : +- CometColumnarExchange (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * CometColumnarToRow (105) - : : : : : : : +- CometFilter (104) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (159) - +- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- * Project (155) - +- * BroadcastHashJoin Inner BuildRight (154) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * Project (146) - : : +- * BroadcastHashJoin Inner BuildRight (145) - : : :- * Project (143) - : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : :- * Project (140) - : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : :- * Project (137) - : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : :- * CometColumnarToRow (134) - : : : : : : +- CometFilter (133) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (132) - : : : : : +- ReusedExchange (135) - : : : : +- ReusedExchange (138) - : : : +- ReusedExchange (141) - : : +- ReusedExchange (144) - : +- ReusedExchange (147) - +- BroadcastExchange (153) - +- * CometColumnarToRow (152) - +- CometFilter (151) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (150) +* CometColumnarToRow (146) ++- CometTakeOrderedAndProject (145) + +- CometUnion (144) + :- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (19) + : : : : +- CometBroadcastHashJoin (18) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) + : : : : +- CometBroadcastExchange (17) + : : : : +- CometFilter (16) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) + : : : +- CometBroadcastExchange (23) + : : : +- CometProject (22) + : : : +- CometFilter (21) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) + : : +- CometBroadcastExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) + :- CometHashAggregate (66) + : +- CometExchange (65) + : +- CometHashAggregate (64) + : +- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometProject (51) + : : : : +- CometBroadcastHashJoin (50) + : : : : :- CometProject (48) + : : : : : +- CometBroadcastHashJoin (47) + : : : : : :- CometProject (45) + : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : :- CometFilter (42) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- ReusedExchange (43) + : : : : : +- ReusedExchange (46) + : : : : +- ReusedExchange (49) + : : : +- CometBroadcastExchange (55) + : : : +- CometProject (54) + : : : +- CometFilter (53) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) + : : +- ReusedExchange (58) + : +- ReusedExchange (61) + :- CometHashAggregate (92) + : +- CometExchange (91) + : +- CometHashAggregate (90) + : +- CometProject (89) + : +- CometBroadcastHashJoin (88) + : :- CometProject (86) + : : +- CometBroadcastHashJoin (85) + : : :- CometProject (83) + : : : +- CometBroadcastHashJoin (82) + : : : :- CometProject (77) + : : : : +- CometBroadcastHashJoin (76) + : : : : :- CometProject (74) + : : : : : +- CometBroadcastHashJoin (73) + : : : : : :- CometProject (71) + : : : : : : +- CometBroadcastHashJoin (70) + : : : : : : :- CometFilter (68) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) + : : : : : : +- ReusedExchange (69) + : : : : : +- ReusedExchange (72) + : : : : +- ReusedExchange (75) + : : : +- CometBroadcastExchange (81) + : : : +- CometProject (80) + : : : +- CometFilter (79) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) + : : +- ReusedExchange (84) + : +- ReusedExchange (87) + :- CometHashAggregate (118) + : +- CometExchange (117) + : +- CometHashAggregate (116) + : +- CometProject (115) + : +- CometBroadcastHashJoin (114) + : :- CometProject (112) + : : +- CometBroadcastHashJoin (111) + : : :- CometProject (109) + : : : +- CometBroadcastHashJoin (108) + : : : :- CometProject (103) + : : : : +- CometBroadcastHashJoin (102) + : : : : :- CometProject (100) + : : : : : +- CometBroadcastHashJoin (99) + : : : : : :- CometProject (97) + : : : : : : +- CometBroadcastHashJoin (96) + : : : : : : :- CometFilter (94) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (98) + : : : : +- ReusedExchange (101) + : : : +- CometBroadcastExchange (107) + : : : +- CometProject (106) + : : : +- CometFilter (105) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) + : : +- ReusedExchange (110) + : +- ReusedExchange (113) + +- CometHashAggregate (143) + +- CometExchange (142) + +- CometHashAggregate (141) + +- CometProject (140) + +- CometBroadcastHashJoin (139) + :- CometProject (135) + : +- CometBroadcastHashJoin (134) + : :- CometProject (132) + : : +- CometBroadcastHashJoin (131) + : : :- CometProject (129) + : : : +- CometBroadcastHashJoin (128) + : : : :- CometProject (126) + : : : : +- CometBroadcastHashJoin (125) + : : : : :- CometProject (123) + : : : : : +- CometBroadcastHashJoin (122) + : : : : : :- CometFilter (120) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) + : : : : : +- ReusedExchange (121) + : : : : +- ReusedExchange (124) + : : : +- ReusedExchange (127) + : : +- ReusedExchange (130) + : +- ReusedExchange (133) + +- CometBroadcastExchange (138) + +- CometFilter (137) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -176,777 +159,688 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [cd_demo_sk#20] -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(26) CometProject +(22) CometProject Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange +(23) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +(25) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(31) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#26] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#26, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) + +(28) CometProject +Input [2]: [d_date_sk#26, d_year#27] +Arguments: [d_date_sk#26], [d_date_sk#26] + +(29) CometBroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Right output [1]: [d_date_sk#26] +Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +(31) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#28, i_item_id#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(42) CometColumnarExchange -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(44) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +(33) CometFilter +Input [2]: [i_item_sk#28, i_item_id#29] +Condition : isnotnull(i_item_sk#28) + +(34) CometProject +Input [2]: [i_item_sk#28, i_item_id#29] +Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] + +(35) CometBroadcastExchange +Input [2]: [i_item_sk#28, i_item_id#30] +Arguments: [i_item_sk#28, i_item_id#30] + +(36) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] +Right output [2]: [i_item_sk#28, i_item_id#30] +Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight + +(37) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] +Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] + +(38) CometHashAggregate +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] +Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] + +(39) CometExchange +Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(40) CometHashAggregate +Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] +Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] + +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(46) CometFilter -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) +(42) CometFilter +Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) -(47) CometColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] +(43) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#66, cd_dep_count#67] -(48) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#93, cd_dep_count#94] +(44) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] +Right output [2]: [cd_demo_sk#66, cd_dep_count#67] +Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#93] -Join type: Inner -Join condition: None +(45) CometProject +Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] +Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -(50) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#93, cd_dep_count#94] +(46) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -(51) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +(47) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] +Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#95] -Join type: Inner -Join condition: None +(48) CometProject +Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -(53) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98] +(49) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#72] -(54) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#99] +(50) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] +Right output [1]: [cd_demo_sk#72] +Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight -(55) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#96] -Right keys [1]: [cd_demo_sk#99] -Join type: Inner -Join condition: None +(51) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -(56) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#98, cd_demo_sk#99] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(58) CometFilter -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) - -(59) CometProject -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Arguments: [ca_address_sk#100, ca_state#25, ca_country#102], [ca_address_sk#100, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#101, 2, true, false, true) AS ca_state#25, ca_country#102] - -(60) CometColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] - -(61) BroadcastExchange -Input [3]: [ca_address_sk#100, ca_state#25, ca_country#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#97] -Right keys [1]: [ca_address_sk#100] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_current_addr_sk#97, c_birth_year#98, ca_address_sk#100, ca_state#25, ca_country#102] - -(64) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#103] - -(65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, d_date_sk#103] - -(67) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#104, i_item_id#29] - -(68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#104] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [10]: [i_item_id#29, ca_country#102, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#105, cast(cs_list_price#87 as decimal(12,2)) AS agg2#106, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#107, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#108, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#109, cast(c_birth_year#98 as decimal(12,2)) AS agg6#110, cast(cd_dep_count#94 as decimal(12,2)) AS agg7#111] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#94, c_birth_year#98, ca_state#25, ca_country#102, i_item_sk#104, i_item_id#29] - -(70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#29, ca_country#102, ca_state#25, agg1#105, agg2#106, agg3#107, agg4#108, agg5#109, agg6#110, agg7#111] -Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] -Functions [7]: [partial_avg(agg1#105), partial_avg(agg2#106), partial_avg(agg3#107), partial_avg(agg4#108), partial_avg(agg5#109), partial_avg(agg6#110), partial_avg(agg7#111)] -Aggregate Attributes [14]: [sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] -Results [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] - -(71) CometColumnarExchange -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] -Arguments: hashpartitioning(i_item_id#29, ca_country#102, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometColumnarToRow [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] - -(73) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#102, ca_state#25, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] -Keys [3]: [i_item_id#29, ca_country#102, ca_state#25] -Functions [7]: [avg(agg1#105), avg(agg2#106), avg(agg3#107), avg(agg4#108), avg(agg5#109), avg(agg6#110), avg(agg7#111)] -Aggregate Attributes [7]: [avg(agg1#105)#140, avg(agg2#106)#141, avg(agg3#107)#142, avg(agg4#108)#143, avg(agg5#109)#144, avg(agg6#110)#145, avg(agg7#111)#146] -Results [11]: [i_item_id#29, ca_country#102, ca_state#25, null AS county#147, avg(agg1#105)#140 AS agg1#148, avg(agg2#106)#141 AS agg2#149, avg(agg3#107)#142 AS agg3#150, avg(agg4#108)#143 AS agg4#151, avg(agg5#109)#144 AS agg5#152, avg(agg6#110)#145 AS agg6#153, avg(agg7#111)#146 AS agg7#154] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +(53) CometFilter +Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) + +(54) CometProject +Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] +Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] + +(55) CometBroadcastExchange +Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] + +(56) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] +Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight + +(57) CometProject +Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] + +(58) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#76] + +(59) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] +Right output [1]: [d_date_sk#76] +Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight + +(60) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] +Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] + +(61) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#77, i_item_id#30] + +(62) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] +Right output [2]: [i_item_sk#77, i_item_id#30] +Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight + +(63) CometProject +Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] +Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] + +(64) CometHashAggregate +Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] +Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] + +(65) CometExchange +Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(66) CometHashAggregate +Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] +Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] + +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#163), dynamicpruningexpression(cs_sold_date_sk#163 IN dynamicpruning#164)] +PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(75) CometFilter -Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] -Condition : ((isnotnull(cs_bill_cdemo_sk#156) AND isnotnull(cs_bill_customer_sk#155)) AND isnotnull(cs_item_sk#157)) - -(76) CometColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163] +(68) CometFilter +Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) -(77) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#165, cd_dep_count#166] +(69) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#109, cd_dep_count#110] -(78) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#156] -Right keys [1]: [cd_demo_sk#165] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] +Right output [2]: [cd_demo_sk#109, cd_dep_count#110] +Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight -(79) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166] -Input [11]: [cs_bill_customer_sk#155, cs_bill_cdemo_sk#156, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_demo_sk#165, cd_dep_count#166] +(71) CometProject +Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] +Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -(80) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +(72) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#155] -Right keys [1]: [c_customer_sk#167] -Join type: Inner -Join condition: None +(73) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] +Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight -(82) Project [codegen id : 23] -Output [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -Input [13]: [cs_bill_customer_sk#155, cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] +(74) CometProject +Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -(83) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#171] +(75) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#115] -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#168] -Right keys [1]: [cd_demo_sk#171] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] +Right output [1]: [cd_demo_sk#115] +Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight -(85) Project [codegen id : 23] -Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] -Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] +(77) CometProject +Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#172, ca_state#173, ca_country#174] +(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(87) CometFilter -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#173, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#172)) - -(88) CometProject -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] - -(89) CometColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#172, ca_country#174] - -(90) BroadcastExchange -Input [2]: [ca_address_sk#172, ca_country#174] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(91) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#169] -Right keys [1]: [ca_address_sk#172] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 23] -Output [10]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [12]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] - -(93) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#175] - -(94) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#163] -Right keys [1]: [d_date_sk#175] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 23] -Output [9]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cs_sold_date_sk#163, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] - -(96) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#176, i_item_id#29] - -(97) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#157] -Right keys [1]: [i_item_sk#176] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 23] -Output [9]: [i_item_id#29, ca_country#174, cast(cs_quantity#158 as decimal(12,2)) AS agg1#177, cast(cs_list_price#159 as decimal(12,2)) AS agg2#178, cast(cs_coupon_amt#161 as decimal(12,2)) AS agg3#179, cast(cs_sales_price#160 as decimal(12,2)) AS agg4#180, cast(cs_net_profit#162 as decimal(12,2)) AS agg5#181, cast(c_birth_year#170 as decimal(12,2)) AS agg6#182, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#183] -Input [11]: [cs_item_sk#157, cs_quantity#158, cs_list_price#159, cs_sales_price#160, cs_coupon_amt#161, cs_net_profit#162, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#29] - -(99) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#29, ca_country#174, agg1#177, agg2#178, agg3#179, agg4#180, agg5#181, agg6#182, agg7#183] -Keys [2]: [i_item_id#29, ca_country#174] -Functions [7]: [partial_avg(agg1#177), partial_avg(agg2#178), partial_avg(agg3#179), partial_avg(agg4#180), partial_avg(agg5#181), partial_avg(agg6#182), partial_avg(agg7#183)] -Aggregate Attributes [14]: [sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197] -Results [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] - -(100) CometColumnarExchange -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] -Arguments: hashpartitioning(i_item_id#29, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(101) CometColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] - -(102) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#174, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211] -Keys [2]: [i_item_id#29, ca_country#174] -Functions [7]: [avg(agg1#177), avg(agg2#178), avg(agg3#179), avg(agg4#180), avg(agg5#181), avg(agg6#182), avg(agg7#183)] -Aggregate Attributes [7]: [avg(agg1#177)#212, avg(agg2#178)#213, avg(agg3#179)#214, avg(agg4#180)#215, avg(agg5#181)#216, avg(agg6#182)#217, avg(agg7#183)#218] -Results [11]: [i_item_id#29, ca_country#174, null AS ca_state#219, null AS county#220, avg(agg1#177)#212 AS agg1#221, avg(agg2#178)#213 AS agg2#222, avg(agg3#179)#214 AS agg3#223, avg(agg4#180)#215 AS agg4#224, avg(agg5#181)#216 AS agg5#225, avg(agg6#182)#217 AS agg6#226, avg(agg7#183)#218 AS agg7#227] - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +(79) CometFilter +Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) + +(80) CometProject +Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] +Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] + +(81) CometBroadcastExchange +Input [2]: [ca_address_sk#116, ca_country#118] +Arguments: [ca_address_sk#116, ca_country#118] + +(82) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] +Right output [2]: [ca_address_sk#116, ca_country#118] +Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight + +(83) CometProject +Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] + +(84) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#119] + +(85) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] +Right output [1]: [d_date_sk#119] +Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight + +(86) CometProject +Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] +Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] + +(87) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#120, i_item_id#30] + +(88) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] +Right output [2]: [i_item_sk#120, i_item_id#30] +Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight + +(89) CometProject +Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] +Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] + +(90) CometHashAggregate +Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] +Keys [2]: [i_item_id#30, ca_country#118] +Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] + +(91) CometExchange +Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] +Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(92) CometHashAggregate +Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] +Keys [2]: [i_item_id#30, ca_country#118] +Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] + +(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#236), dynamicpruningexpression(cs_sold_date_sk#236 IN dynamicpruning#237)] +PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) CometFilter -Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] -Condition : ((isnotnull(cs_bill_cdemo_sk#229) AND isnotnull(cs_bill_customer_sk#228)) AND isnotnull(cs_item_sk#230)) +(94) CometFilter +Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) -(105) CometColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236] +(95) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#152, cd_dep_count#153] -(106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#238, cd_dep_count#239] +(96) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] +Right output [2]: [cd_demo_sk#152, cd_dep_count#153] +Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#229] -Right keys [1]: [cd_demo_sk#238] -Join type: Inner -Join condition: None +(97) CometProject +Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] +Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -(108) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239] -Input [11]: [cs_bill_customer_sk#228, cs_bill_cdemo_sk#229, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_demo_sk#238, cd_dep_count#239] +(98) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -(109) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +(99) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] +Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight -(110) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#228] -Right keys [1]: [c_customer_sk#240] -Join type: Inner -Join condition: None +(100) CometProject +Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -(111) Project [codegen id : 31] -Output [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -Input [13]: [cs_bill_customer_sk#228, cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] +(101) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#158] -(112) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#244] +(102) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] +Right output [1]: [cd_demo_sk#158] +Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight -(113) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#241] -Right keys [1]: [cd_demo_sk#244] -Join type: Inner -Join condition: None +(103) CometProject +Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -(114) Project [codegen id : 31] -Output [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] -Input [12]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] - -(115) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#245, ca_state#246] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#159, ca_state#160] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(116) CometFilter -Input [2]: [ca_address_sk#245, ca_state#246] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#246, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#245)) - -(117) CometProject -Input [2]: [ca_address_sk#245, ca_state#246] -Arguments: [ca_address_sk#245], [ca_address_sk#245] - -(118) CometColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#245] - -(119) BroadcastExchange -Input [1]: [ca_address_sk#245] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#242] -Right keys [1]: [ca_address_sk#245] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [9]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243] -Input [11]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] - -(122) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#247] - -(123) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#236] -Right keys [1]: [d_date_sk#247] -Join type: Inner -Join condition: None - -(124) Project [codegen id : 31] -Output [8]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243] -Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cs_sold_date_sk#236, cd_dep_count#239, c_birth_year#243, d_date_sk#247] - -(125) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#248, i_item_id#29] - -(126) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#230] -Right keys [1]: [i_item_sk#248] -Join type: Inner -Join condition: None - -(127) Project [codegen id : 31] -Output [8]: [i_item_id#29, cast(cs_quantity#231 as decimal(12,2)) AS agg1#249, cast(cs_list_price#232 as decimal(12,2)) AS agg2#250, cast(cs_coupon_amt#234 as decimal(12,2)) AS agg3#251, cast(cs_sales_price#233 as decimal(12,2)) AS agg4#252, cast(cs_net_profit#235 as decimal(12,2)) AS agg5#253, cast(c_birth_year#243 as decimal(12,2)) AS agg6#254, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#255] -Input [10]: [cs_item_sk#230, cs_quantity#231, cs_list_price#232, cs_sales_price#233, cs_coupon_amt#234, cs_net_profit#235, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#29] - -(128) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#29, agg1#249, agg2#250, agg3#251, agg4#252, agg5#253, agg6#254, agg7#255] -Keys [1]: [i_item_id#29] -Functions [7]: [partial_avg(agg1#249), partial_avg(agg2#250), partial_avg(agg3#251), partial_avg(agg4#252), partial_avg(agg5#253), partial_avg(agg6#254), partial_avg(agg7#255)] -Aggregate Attributes [14]: [sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Results [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] - -(129) CometColumnarExchange -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(130) CometColumnarToRow [codegen id : 32] -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] - -(131) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#29, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281, sum#282, count#283] -Keys [1]: [i_item_id#29] -Functions [7]: [avg(agg1#249), avg(agg2#250), avg(agg3#251), avg(agg4#252), avg(agg5#253), avg(agg6#254), avg(agg7#255)] -Aggregate Attributes [7]: [avg(agg1#249)#284, avg(agg2#250)#285, avg(agg3#251)#286, avg(agg4#252)#287, avg(agg5#253)#288, avg(agg6#254)#289, avg(agg7#255)#290] -Results [11]: [i_item_id#29, null AS ca_country#291, null AS ca_state#292, null AS county#293, avg(agg1#249)#284 AS agg1#294, avg(agg2#250)#285 AS agg2#295, avg(agg3#251)#286 AS agg3#296, avg(agg4#252)#287 AS agg4#297, avg(agg5#253)#288 AS agg5#298, avg(agg6#254)#289 AS agg6#299, avg(agg7#255)#290 AS agg7#300] - -(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +(105) CometFilter +Input [2]: [ca_address_sk#159, ca_state#160] +Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) + +(106) CometProject +Input [2]: [ca_address_sk#159, ca_state#160] +Arguments: [ca_address_sk#159], [ca_address_sk#159] + +(107) CometBroadcastExchange +Input [1]: [ca_address_sk#159] +Arguments: [ca_address_sk#159] + +(108) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] +Right output [1]: [ca_address_sk#159] +Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight + +(109) CometProject +Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] + +(110) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#161] + +(111) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] +Right output [1]: [d_date_sk#161] +Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight + +(112) CometProject +Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] +Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] + +(113) ReusedExchange [Reuses operator id: 35] +Output [2]: [i_item_sk#162, i_item_id#30] + +(114) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] +Right output [2]: [i_item_sk#162, i_item_id#30] +Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight + +(115) CometProject +Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] +Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] + +(116) CometHashAggregate +Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] +Keys [1]: [i_item_id#30] +Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] + +(117) CometExchange +Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(118) CometHashAggregate +Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] +Keys [1]: [i_item_id#30] +Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] + +(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#309), dynamicpruningexpression(cs_sold_date_sk#309 IN dynamicpruning#310)] +PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(133) CometFilter -Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] -Condition : ((isnotnull(cs_bill_cdemo_sk#302) AND isnotnull(cs_bill_customer_sk#301)) AND isnotnull(cs_item_sk#303)) +(120) CometFilter +Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) -(134) CometColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309] +(121) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#194, cd_dep_count#195] -(135) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#311, cd_dep_count#312] +(122) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] +Right output [2]: [cd_demo_sk#194, cd_dep_count#195] +Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight -(136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#302] -Right keys [1]: [cd_demo_sk#311] -Join type: Inner -Join condition: None +(123) CometProject +Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] +Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -(137) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312] -Input [11]: [cs_bill_customer_sk#301, cs_bill_cdemo_sk#302, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_demo_sk#311, cd_dep_count#312] +(124) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -(138) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +(125) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] +Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight -(139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#301] -Right keys [1]: [c_customer_sk#313] -Join type: Inner -Join condition: None +(126) CometProject +Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -(140) Project [codegen id : 39] -Output [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -Input [13]: [cs_bill_customer_sk#301, cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] +(127) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#200] -(141) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#317] +(128) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] +Right output [1]: [cd_demo_sk#200] +Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight -(142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#314] -Right keys [1]: [cd_demo_sk#317] -Join type: Inner -Join condition: None +(129) CometProject +Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -(143) Project [codegen id : 39] -Output [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] -Input [12]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] +(130) ReusedExchange [Reuses operator id: 107] +Output [1]: [ca_address_sk#201] -(144) ReusedExchange [Reuses operator id: 119] -Output [1]: [ca_address_sk#318] +(131) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] +Right output [1]: [ca_address_sk#201] +Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight -(145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#315] -Right keys [1]: [ca_address_sk#318] -Join type: Inner -Join condition: None +(132) CometProject +Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -(146) Project [codegen id : 39] -Output [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316] -Input [11]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] +(133) ReusedExchange [Reuses operator id: 29] +Output [1]: [d_date_sk#202] -(147) ReusedExchange [Reuses operator id: 168] -Output [1]: [d_date_sk#319] +(134) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] +Right output [1]: [d_date_sk#202] +Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight -(148) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#309] -Right keys [1]: [d_date_sk#319] -Join type: Inner -Join condition: None +(135) CometProject +Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] +Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -(149) Project [codegen id : 39] -Output [8]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316] -Input [10]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cs_sold_date_sk#309, cd_dep_count#312, c_birth_year#316, d_date_sk#319] - -(150) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#320] +(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#203] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(151) CometFilter -Input [1]: [i_item_sk#320] -Condition : isnotnull(i_item_sk#320) - -(152) CometColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#320] +(137) CometFilter +Input [1]: [i_item_sk#203] +Condition : isnotnull(i_item_sk#203) -(153) BroadcastExchange -Input [1]: [i_item_sk#320] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(138) CometBroadcastExchange +Input [1]: [i_item_sk#203] +Arguments: [i_item_sk#203] -(154) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#303] -Right keys [1]: [i_item_sk#320] -Join type: Inner -Join condition: None +(139) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] +Right output [1]: [i_item_sk#203] +Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight -(155) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#304 as decimal(12,2)) AS agg1#321, cast(cs_list_price#305 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#307 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#306 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#308 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] -Input [9]: [cs_item_sk#303, cs_quantity#304, cs_list_price#305, cs_sales_price#306, cs_coupon_amt#307, cs_net_profit#308, cd_dep_count#312, c_birth_year#316, i_item_sk#320] +(140) CometProject +Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] +Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] -(156) HashAggregate [codegen id : 39] -Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] +(141) CometHashAggregate +Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] Keys: [] -Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] -Aggregate Attributes [14]: [sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339, sum#340, count#341] -Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] -(157) CometColumnarExchange -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(142) CometExchange +Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(158) CometColumnarToRow [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] - -(159) HashAggregate [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] +(143) CometHashAggregate +Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] Keys: [] -Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] -Aggregate Attributes [7]: [avg(agg1#321)#356, avg(agg2#322)#357, avg(agg3#323)#358, avg(agg4#324)#359, avg(agg5#325)#360, avg(agg6#326)#361, avg(agg7#327)#362] -Results [11]: [null AS i_item_id#363, null AS ca_country#364, null AS ca_state#365, null AS county#366, avg(agg1#321)#356 AS agg1#367, avg(agg2#322)#357 AS agg2#368, avg(agg3#323)#358 AS agg3#369, avg(agg4#324)#359 AS agg4#370, avg(agg5#325)#360 AS agg5#371, avg(agg6#326)#361 AS agg6#372, avg(agg7#327)#362 AS agg7#373] - -(160) Union +Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] -(161) RowToColumnar -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(144) CometUnion +Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] +Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] +Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] +Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] -(162) CometTakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#31 ASC NULLS FIRST,ca_state#32 ASC NULLS FIRST,ca_county#33 ASC NULLS FIRST,i_item_id#30 ASC NULLS FIRST], output=[i_item_id#30,ca_country#31,ca_state#32,ca_county#33,agg1#76,agg2#77,agg3#78,agg4#79,agg5#80,agg6#81,agg7#82]), [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82], 100, 0, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(145) CometTakeOrderedAndProject +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -(163) CometColumnarToRow [codegen id : 41] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +(146) CometColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (168) -+- * CometColumnarToRow (167) - +- CometProject (166) - +- CometFilter (165) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (164) +BroadcastExchange (151) ++- * CometColumnarToRow (150) + +- CometProject (149) + +- CometFilter (148) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) -(164) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#374] +(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(165) CometFilter -Input [2]: [d_date_sk#26, d_year#374] -Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_date_sk#26)) +(148) CometFilter +Input [2]: [d_date_sk#26, d_year#27] +Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) -(166) CometProject -Input [2]: [d_date_sk#26, d_year#374] +(149) CometProject +Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(167) CometColumnarToRow [codegen id : 1] +(150) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(168) BroadcastExchange +(151) BroadcastExchange Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#163 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#236 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#309 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt index 7a8f23c282..b18a444bb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt @@ -1,255 +1,214 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 103 out of 210 eligible operators (49%). Final plan contains 43 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index 91800fcca0..1572a2a240 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt @@ -1,245 +1,160 @@ -WholeStageCodegen (41) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - RowToColumnar - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_county,ca_state,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange [cd_demo_sk] #5 + CometFilter [cd_demo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_item_id] #8 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country,ca_state] #9 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_state,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10 + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id,ca_country] #11 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk,ca_country] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk,ca_country] #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange [i_item_id] #13 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange [ca_address_sk] #14 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometExchange #15 + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_demo_sk,cd_dep_count] + CometFilter [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + ReusedExchange [ca_address_sk] #14 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [i_item_sk] #16 + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt index d8ae1972ba..322bbe63b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt @@ -1,29 +1,27 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- RowToColumnar (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -38,134 +36,129 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(15) CometColumnarExchange +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) RowToColumnar -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(24) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(25) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +(26) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index a2cde6785b..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -1,37 +1,31 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt index 5a0729e0da..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt @@ -1,47 +1,35 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - RowToColumnar - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt index d8ae1972ba..322bbe63b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt @@ -1,29 +1,27 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- RowToColumnar (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -38,134 +36,129 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(15) CometColumnarExchange +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) RowToColumnar -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(24) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +(25) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +(26) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt index a2cde6785b..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt @@ -1,37 +1,31 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 15 out of 27 eligible operators (55%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt index 5a0729e0da..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt @@ -1,47 +1,35 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - RowToColumnar - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt index 295d35ec0a..f85c898208 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt @@ -1,30 +1,30 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- RowToColumnar (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastNestedLoopJoin Inner BuildRight (17) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * CometColumnarToRow (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * CometColumnarToRow (24) + +- CometColumnarExchange (23) + +- * HashAggregate (22) + +- * Expand (21) + +- * Project (20) + +- * BroadcastNestedLoopJoin Inner BuildRight (19) + :- * CometColumnarToRow (15) + : +- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + +- BroadcastExchange (18) + +- * CometColumnarToRow (17) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -39,109 +39,114 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) CometColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) -(9) CometProject -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] +(11) CometProject +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(12) CometBroadcastExchange +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(11) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(14) CometProject +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(13) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(15) CometColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(15) CometColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 1] Input: [] -(16) BroadcastExchange +(18) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] +Arguments: IdentityBroadcastMode, [plan_id=1] -(17) BroadcastNestedLoopJoin [codegen id : 4] +(19) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(20) Project [codegen id : 2] +Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] +Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(21) Expand [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] +Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(22) HashAggregate [codegen id : 2] +Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Aggregate Attributes [2]: [sum#21, count#22] +Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(21) CometColumnarExchange -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(23) CometColumnarExchange +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +(24) CometColumnarToRow [codegen id : 3] +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(23) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(25) HashAggregate [codegen id : 3] +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] -Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] - -(24) RowToColumnar -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] - -(25) CometTakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#15 ASC NULLS FIRST,i_brand#16 ASC NULLS FIRST,i_class#17 ASC NULLS FIRST,i_category#18 ASC NULLS FIRST], output=[i_product_name#15,i_brand#16,i_class#17,i_category#18,qoh#25]), [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] +Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] -(26) CometColumnarToRow [codegen id : 6] -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +(26) TakeOrderedAndProject +Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] +Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] ===== Subqueries ===== @@ -154,18 +159,18 @@ BroadcastExchange (31) (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#26] +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#26] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) (29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#26] +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] (30) CometColumnarToRow [codegen id : 1] @@ -173,6 +178,6 @@ Input [1]: [d_date_sk#5] (31) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index d5e580d4a7..261903252b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -1,38 +1,34 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + TakeOrderedAndProject [COMET: ] ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + +- CometColumnarToRow + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt index 00be4f0608..221c6063ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt @@ -1,47 +1,41 @@ -WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (2) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + CometColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt index 295d35ec0a..f85c898208 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt @@ -1,30 +1,30 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- RowToColumnar (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastNestedLoopJoin Inner BuildRight (17) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * CometColumnarToRow (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- * CometColumnarToRow (24) + +- CometColumnarExchange (23) + +- * HashAggregate (22) + +- * Expand (21) + +- * Project (20) + +- * BroadcastNestedLoopJoin Inner BuildRight (19) + :- * CometColumnarToRow (15) + : +- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + +- BroadcastExchange (18) + +- * CometColumnarToRow (17) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -39,109 +39,114 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) CometColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) -(9) CometProject -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] +(11) CometProject +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(12) CometBroadcastExchange +Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(11) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(14) CometProject +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(13) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(15) CometColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(15) CometColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 1] Input: [] -(16) BroadcastExchange +(18) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] +Arguments: IdentityBroadcastMode, [plan_id=1] -(17) BroadcastNestedLoopJoin [codegen id : 4] +(19) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] +(20) Project [codegen id : 2] +Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] +Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(21) Expand [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] +Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(22) HashAggregate [codegen id : 2] +Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] +Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +Aggregate Attributes [2]: [sum#21, count#22] +Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(21) CometColumnarExchange -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(23) CometColumnarExchange +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] +(24) CometColumnarToRow [codegen id : 3] +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(23) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] +(25) HashAggregate [codegen id : 3] +Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] +Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] -Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] - -(24) RowToColumnar -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] - -(25) CometTakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#15 ASC NULLS FIRST,i_brand#16 ASC NULLS FIRST,i_class#17 ASC NULLS FIRST,i_category#18 ASC NULLS FIRST], output=[i_product_name#15,i_brand#16,i_class#17,i_category#18,qoh#25]), [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] +Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] -(26) CometColumnarToRow [codegen id : 6] -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] +(26) TakeOrderedAndProject +Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] +Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] ===== Subqueries ===== @@ -154,18 +159,18 @@ BroadcastExchange (31) (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#26] +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct (28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#26] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) (29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#26] +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] (30) CometColumnarToRow [codegen id : 1] @@ -173,6 +178,6 @@ Input [1]: [d_date_sk#5] (31) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt index d5e580d4a7..261903252b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt @@ -1,38 +1,34 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + TakeOrderedAndProject [COMET: ] ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + +- CometColumnarToRow + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 00be4f0608..221c6063ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -1,47 +1,41 @@ -WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - RowToColumnar - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (2) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + CometColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt index 34bb4d3071..bf16cc4d21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt @@ -1,61 +1,51 @@ == Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- RowToColumnar (55) - +- Union (54) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) - :- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) - : +- ReusedExchange (33) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- ReusedExchange (40) - +- * HashAggregate (53) - +- * CometColumnarToRow (52) - +- CometColumnarExchange (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- ReusedExchange (47) +* CometColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometUnion (45) + :- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) + :- CometHashAggregate (29) + : +- CometExchange (28) + : +- CometHashAggregate (27) + : +- CometHashAggregate (26) + : +- ReusedExchange (25) + :- CometHashAggregate (34) + : +- CometExchange (33) + : +- CometHashAggregate (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + :- CometHashAggregate (39) + : +- CometExchange (38) + : +- CometHashAggregate (37) + : +- CometHashAggregate (36) + : +- ReusedExchange (35) + +- CometHashAggregate (44) + +- CometExchange (43) + +- CometHashAggregate (42) + +- CometHashAggregate (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -70,295 +60,242 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) + +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] -(4) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#6] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(11) CometProject +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(14) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) +(16) CometFilter +Input [1]: [w_warehouse_sk#17] +Condition : isnotnull(w_warehouse_sk#17) -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] +(17) CometBroadcastExchange +Input [1]: [w_warehouse_sk#17] +Arguments: [w_warehouse_sk#17] -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Right output [1]: [w_warehouse_sk#17] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] +Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +(20) CometHashAggregate +Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#17, count#18] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(21) CometColumnarExchange -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +(21) CometExchange +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +(22) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] -Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] - -(24) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] - -(25) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#31] -Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] - -(26) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(27) CometColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(28) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] + +(23) CometHashAggregate +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] +Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] +Functions [1]: [partial_avg(qoh#24)] + +(24) CometHashAggregate +Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] +Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] +Functions [1]: [avg(qoh#24)] + +(25) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] + +(26) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#29)] + +(27) CometHashAggregate +Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Functions [1]: [partial_avg(qoh#30)] + +(28) CometExchange +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Functions [1]: [avg(qoh#30)] + +(30) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] + +(31) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] -Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] -(29) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +(32) CometHashAggregate +Input [3]: [i_product_name#16, i_brand#13, qoh#36] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [partial_avg(qoh#36)] -Aggregate Attributes [2]: [sum#37, count#38] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] - -(30) CometColumnarExchange -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +(33) CometExchange +Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +(34) CometHashAggregate +Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [avg(qoh#36)] -Aggregate Attributes [1]: [avg(qoh#36)#41] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] - -(33) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(34) CometColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(35) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] -Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] - -(36) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#15, i_brand#12, qoh#47] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(37) CometColumnarExchange -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(39) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(41) CometColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#58)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] -Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] - -(43) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#15, qoh#59] -Keys [1]: [i_product_name#15] -Functions [1]: [partial_avg(qoh#59)] -Aggregate Attributes [2]: [sum#60, count#61] -Results [3]: [i_product_name#15, sum#62, count#63] - -(44) CometColumnarExchange -Input [3]: [i_product_name#15, sum#62, count#63] -Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] - -(46) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] -Keys [1]: [i_product_name#15] -Functions [1]: [avg(qoh#59)] -Aggregate Attributes [1]: [avg(qoh#59)#64] -Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS i_category#67, avg(qoh#59)#64 AS qoh#68] - -(47) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(48) CometColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(49) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#71)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] -Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] - -(50) HashAggregate [codegen id : 28] -Input [1]: [qoh#72] -Keys: [] -Functions [1]: [partial_avg(qoh#72)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] +(35) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -(51) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(36) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#41)] -(52) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#75, count#76] +(37) CometHashAggregate +Input [2]: [i_product_name#16, qoh#42] +Keys [1]: [i_product_name#16] +Functions [1]: [partial_avg(qoh#42)] -(53) HashAggregate [codegen id : 29] -Input [2]: [sum#75, count#76] +(38) CometExchange +Input [3]: [i_product_name#16, sum#43, count#44] +Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(39) CometHashAggregate +Input [3]: [i_product_name#16, sum#43, count#44] +Keys [1]: [i_product_name#16] +Functions [1]: [avg(qoh#42)] + +(40) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] + +(41) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#47)] + +(42) CometHashAggregate +Input [1]: [qoh#48] Keys: [] -Functions [1]: [avg(qoh#72)] -Aggregate Attributes [1]: [avg(qoh#72)#77] -Results [5]: [null AS i_product_name#78, null AS i_brand#79, null AS i_class#80, null AS i_category#81, avg(qoh#72)#77 AS qoh#82] +Functions [1]: [partial_avg(qoh#48)] + +(43) CometExchange +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(54) Union +(44) CometHashAggregate +Input [2]: [sum#49, count#50] +Keys: [] +Functions [1]: [avg(qoh#48)] -(55) RowToColumnar -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(45) CometUnion +Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] +Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] +Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] +Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] -(56) CometTakeOrderedAndProject -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#32 ASC NULLS FIRST,i_product_name#22 ASC NULLS FIRST,i_brand#23 ASC NULLS FIRST,i_class#24 ASC NULLS FIRST,i_category#25 ASC NULLS FIRST], output=[i_product_name#22,i_brand#23,i_class#24,i_category#25,qoh#32]), [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32], 100, 0, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(46) CometTakeOrderedAndProject +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -(57) CometColumnarToRow [codegen id : 30] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(47) CometColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#83] +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#83] -Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_month_seq#83 <= 1223)) AND isnotnull(d_date_sk#6)) +(49) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(60) CometProject -Input [2]: [d_date_sk#6, d_month_seq#83] +(50) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(61) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(62) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index 08f1fe4f48..c018673888 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -1,189 +1,159 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 75 out of 151 eligible operators (49%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt index d855717f2e..b59605103e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt @@ -1,94 +1,57 @@ -WholeStageCodegen (30) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - RowToColumnar - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] + CometUnion [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] + CometExchange [i_product_name,i_brand,i_class,i_category] #1 + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #5 + CometFilter [w_warehouse_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name,i_brand,i_class] #6 + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] + CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name,i_brand] #7 + CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] + CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name] #8 + CometHashAggregate [qoh] [i_product_name,sum,count] + CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange #9 + CometHashAggregate [qoh] [sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt index 34bb4d3071..bf16cc4d21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt @@ -1,61 +1,51 @@ == Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- RowToColumnar (55) - +- Union (54) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (14) - :- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) - : +- ReusedExchange (33) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- ReusedExchange (40) - +- * HashAggregate (53) - +- * CometColumnarToRow (52) - +- CometColumnarExchange (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- ReusedExchange (47) +* CometColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometUnion (45) + :- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) + :- CometHashAggregate (29) + : +- CometExchange (28) + : +- CometHashAggregate (27) + : +- CometHashAggregate (26) + : +- ReusedExchange (25) + :- CometHashAggregate (34) + : +- CometExchange (33) + : +- CometHashAggregate (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + :- CometHashAggregate (39) + : +- CometExchange (38) + : +- CometHashAggregate (37) + : +- CometHashAggregate (36) + : +- ReusedExchange (35) + +- CometHashAggregate (44) + +- CometExchange (43) + +- CometHashAggregate (42) + +- CometHashAggregate (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory @@ -70,295 +60,242 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) + +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] -(4) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#6] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(11) CometProject +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] +(14) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) +(16) CometFilter +Input [1]: [w_warehouse_sk#17] +Condition : isnotnull(w_warehouse_sk#17) -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] +(17) CometBroadcastExchange +Input [1]: [w_warehouse_sk#17] +Arguments: [w_warehouse_sk#17] -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Right output [1]: [w_warehouse_sk#17] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] +Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +(20) CometHashAggregate +Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#17, count#18] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(21) CometColumnarExchange -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] +(21) CometExchange +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] +(22) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] -Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] - -(24) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] - -(25) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#31] -Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] - -(26) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(27) CometColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(28) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] + +(23) CometHashAggregate +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] +Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] +Functions [1]: [partial_avg(qoh#24)] + +(24) CometHashAggregate +Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] +Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] +Functions [1]: [avg(qoh#24)] + +(25) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] + +(26) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#29)] + +(27) CometHashAggregate +Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Functions [1]: [partial_avg(qoh#30)] + +(28) CometExchange +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Functions [1]: [avg(qoh#30)] + +(30) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] + +(31) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] -Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] -(29) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +(32) CometHashAggregate +Input [3]: [i_product_name#16, i_brand#13, qoh#36] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [partial_avg(qoh#36)] -Aggregate Attributes [2]: [sum#37, count#38] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] - -(30) CometColumnarExchange -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] +(33) CometExchange +Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] +(34) CometHashAggregate +Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [avg(qoh#36)] -Aggregate Attributes [1]: [avg(qoh#36)#41] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] - -(33) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(34) CometColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(35) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] -Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] - -(36) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#15, i_brand#12, qoh#47] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(37) CometColumnarExchange -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(39) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(41) CometColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#58)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] -Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] - -(43) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#15, qoh#59] -Keys [1]: [i_product_name#15] -Functions [1]: [partial_avg(qoh#59)] -Aggregate Attributes [2]: [sum#60, count#61] -Results [3]: [i_product_name#15, sum#62, count#63] - -(44) CometColumnarExchange -Input [3]: [i_product_name#15, sum#62, count#63] -Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] - -(46) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] -Keys [1]: [i_product_name#15] -Functions [1]: [avg(qoh#59)] -Aggregate Attributes [1]: [avg(qoh#59)#64] -Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS i_category#67, avg(qoh#59)#64 AS qoh#68] - -(47) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(48) CometColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(49) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#71)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] -Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] - -(50) HashAggregate [codegen id : 28] -Input [1]: [qoh#72] -Keys: [] -Functions [1]: [partial_avg(qoh#72)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] +(35) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -(51) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(36) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#41)] -(52) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#75, count#76] +(37) CometHashAggregate +Input [2]: [i_product_name#16, qoh#42] +Keys [1]: [i_product_name#16] +Functions [1]: [partial_avg(qoh#42)] -(53) HashAggregate [codegen id : 29] -Input [2]: [sum#75, count#76] +(38) CometExchange +Input [3]: [i_product_name#16, sum#43, count#44] +Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(39) CometHashAggregate +Input [3]: [i_product_name#16, sum#43, count#44] +Keys [1]: [i_product_name#16] +Functions [1]: [avg(qoh#42)] + +(40) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] + +(41) CometHashAggregate +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#47)] + +(42) CometHashAggregate +Input [1]: [qoh#48] Keys: [] -Functions [1]: [avg(qoh#72)] -Aggregate Attributes [1]: [avg(qoh#72)#77] -Results [5]: [null AS i_product_name#78, null AS i_brand#79, null AS i_class#80, null AS i_category#81, avg(qoh#72)#77 AS qoh#82] +Functions [1]: [partial_avg(qoh#48)] + +(43) CometExchange +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(54) Union +(44) CometHashAggregate +Input [2]: [sum#49, count#50] +Keys: [] +Functions [1]: [avg(qoh#48)] -(55) RowToColumnar -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(45) CometUnion +Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] +Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] +Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] +Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] -(56) CometTakeOrderedAndProject -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#32 ASC NULLS FIRST,i_product_name#22 ASC NULLS FIRST,i_brand#23 ASC NULLS FIRST,i_class#24 ASC NULLS FIRST,i_category#25 ASC NULLS FIRST], output=[i_product_name#22,i_brand#23,i_class#24,i_category#25,qoh#32]), [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32], 100, 0, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(46) CometTakeOrderedAndProject +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -(57) CometColumnarToRow [codegen id : 30] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] +(47) CometColumnarToRow [codegen id : 1] +Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#83] +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#83] -Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_month_seq#83 <= 1223)) AND isnotnull(d_date_sk#6)) +(49) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(60) CometProject -Input [2]: [d_date_sk#6, d_month_seq#83] +(50) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(61) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(62) BroadcastExchange +(52) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt index 08f1fe4f48..c018673888 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt @@ -1,189 +1,159 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 75 out of 151 eligible operators (49%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt index d855717f2e..b59605103e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt @@ -1,94 +1,57 @@ -WholeStageCodegen (30) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - RowToColumnar - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] + CometUnion [i_product_name,i_brand,i_class,i_category,qoh] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] + CometExchange [i_product_name,i_brand,i_class,i_category] #1 + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange [w_warehouse_sk] #5 + CometFilter [w_warehouse_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name,i_brand,i_class] #6 + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] + CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name,i_brand] #7 + CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] + CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange [i_product_name] #8 + CometHashAggregate [qoh] [i_product_name,sum,count] + CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometExchange #9 + CometHashAggregate [qoh] [sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt index fa411d777f..17ef709df5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt @@ -1,57 +1,54 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometSort (52) - +- CometColumnarExchange (51) - +- * Filter (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- CometColumnarExchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * CometColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (35) +* CometColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- * Filter (47) + +- * HashAggregate (46) + +- * CometColumnarToRow (45) + +- CometColumnarExchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -109,363 +106,344 @@ Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7 Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(13) CometColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) -(16) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(18) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(20) CometFilter Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) -(23) CometProject +(21) CometProject Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) BroadcastExchange +(22) CometBroadcastExchange Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] +Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight -(27) Project [codegen id : 5] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +(24) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) -(30) CometProject +(27) CometProject Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) BroadcastExchange +(28) CometBroadcastExchange Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#26] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight -(34) Project [codegen id : 5] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(30) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] + +(31) CometColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] ReadSchema: struct -(36) CometFilter +(33) CometFilter Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) -(37) CometProject +(34) CometProject Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(38) CometColumnarToRow [codegen id : 4] +(35) CometColumnarToRow [codegen id : 1] Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) BroadcastExchange +(36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(38) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(42) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#39] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(43) CometColumnarExchange +(40) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 6] +(41) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(45) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(46) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(47) CometColumnarExchange +(44) CometColumnarExchange Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(48) CometColumnarToRow [codegen id : 7] +(45) CometColumnarToRow [codegen id : 4] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(49) HashAggregate [codegen id : 7] +(46) HashAggregate [codegen id : 4] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(50) Filter [codegen id : 7] +(47) Filter [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) -(51) CometColumnarExchange +(48) CometColumnarExchange Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(52) CometSort +(49) CometSort Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(53) CometColumnarToRow [codegen id : 8] +(50) CometColumnarToRow [codegen id : 5] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== -Subquery:1 Hosting operator id = 50 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (84) -+- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * Project (76) - +- * BroadcastHashJoin Inner BuildRight (75) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * CometColumnarToRow (60) - : : : : +- CometProject (59) - : : : : +- CometSortMergeJoin (58) - : : : : :- CometSort (55) - : : : : : +- ReusedExchange (54) - : : : : +- CometSort (57) - : : : : +- ReusedExchange (56) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (68) - : : +- * CometColumnarToRow (67) - : : +- CometProject (66) - : : +- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (64) - : +- ReusedExchange (71) - +- ReusedExchange (74) - - -(54) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (80) ++- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * CometColumnarToRow (69) + : +- CometProject (68) + : +- CometBroadcastHashJoin (67) + : :- CometProject (65) + : : +- CometBroadcastHashJoin (64) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (56) + : : : : +- CometSortMergeJoin (55) + : : : : :- CometSort (52) + : : : : : +- ReusedExchange (51) + : : : : +- CometSort (54) + : : : : +- ReusedExchange (53) + : : : +- ReusedExchange (57) + : : +- CometBroadcastExchange (63) + : : +- CometProject (62) + : : +- CometFilter (61) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) + : +- ReusedExchange (66) + +- ReusedExchange (70) + + +(51) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -(55) CometSort +(52) CometSort Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] -(56) ReusedExchange [Reuses operator id: 9] +(53) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#56, sr_ticket_number#57] -(57) CometSort +(54) CometSort Input [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] -(58) CometSortMergeJoin +(55) CometSortMergeJoin Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Right output [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner -(59) CometProject +(56) CometProject Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -(60) CometColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(61) ReusedExchange [Reuses operator id: 18] +(57) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -(62) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#53] -Right keys [1]: [s_store_sk#58] -Join type: Inner -Join condition: None +(58) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] +Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] +Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight -(63) Project [codegen id : 5] -Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] +(59) CometProject Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] +Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(65) CometFilter +(61) CometFilter Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Condition : isnotnull(i_item_sk#60) -(66) CometProject +(62) CometProject Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] -(67) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(68) BroadcastExchange +(63) CometBroadcastExchange Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(69) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#51] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None +(64) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] +Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight -(70) Project [codegen id : 5] -Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +(65) CometProject Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(71) ReusedExchange [Reuses operator id: 32] +(66) ReusedExchange [Reuses operator id: 28] Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(72) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#66] -Join type: Inner -Join condition: None +(67) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight -(73) Project [codegen id : 5] -Output [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +(68) CometProject Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] + +(69) CometColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(74) ReusedExchange [Reuses operator id: 39] +(70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(75) BroadcastHashJoin [codegen id : 5] +(71) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(76) Project [codegen id : 5] +(72) Project [codegen id : 2] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(77) HashAggregate [codegen id : 5] +(73) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum#71] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(78) CometColumnarExchange +(74) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(79) CometColumnarToRow [codegen id : 6] +(75) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(80) HashAggregate [codegen id : 6] +(76) HashAggregate [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(81) HashAggregate [codegen id : 6] +(77) HashAggregate [codegen id : 3] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] Aggregate Attributes [2]: [sum#74, count#75] Results [2]: [sum#76, count#77] -(82) CometColumnarExchange +(78) CometColumnarExchange Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(83) CometColumnarToRow [codegen id : 7] +(79) CometColumnarToRow [codegen id : 4] Input [2]: [sum#76, count#77] -(84) HashAggregate [codegen id : 7] +(80) HashAggregate [codegen id : 4] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt index 0ed56d0b0d..7847aefd83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt @@ -13,37 +13,34 @@ CometColumnarToRow : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- 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 @@ -62,37 +59,34 @@ CometColumnarToRow +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- 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 @@ -102,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 54 out of 88 eligible operators (61%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt index ce751cb78d..62f492f632 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt @@ -1,76 +1,70 @@ -WholeStageCodegen (8) +WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [paid] Subquery #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] @@ -83,33 +77,21 @@ WholeStageCodegen (8) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (4) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt index fa411d777f..17ef709df5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt @@ -1,57 +1,54 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometSort (52) - +- CometColumnarExchange (51) - +- * Filter (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- CometColumnarExchange (47) - +- * HashAggregate (46) - +- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * CometColumnarToRow (13) - : : : : +- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (35) +* CometColumnarToRow (50) ++- CometSort (49) + +- CometColumnarExchange (48) + +- * Filter (47) + +- * HashAggregate (46) + +- * CometColumnarToRow (45) + +- CometColumnarExchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (9) + : : : : +- CometProject (8) + : : : : +- CometFilter (7) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) + : : : +- CometBroadcastExchange (16) + : : : +- CometProject (15) + : : : +- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : : +- CometBroadcastExchange (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) + : +- CometBroadcastExchange (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) + +- BroadcastExchange (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -109,363 +106,344 @@ Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7 Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -(13) CometColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter +(14) CometFilter Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) -(16) CometProject +(15) CometProject Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(18) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] +(18) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(22) CometFilter +(20) CometFilter Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) -(23) CometProject +(21) CometProject Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) BroadcastExchange +(22) CometBroadcastExchange Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] +Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight -(27) Project [codegen id : 5] -Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +(24) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(29) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) -(30) CometProject +(27) CometProject Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) BroadcastExchange +(28) CometBroadcastExchange Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#26] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] +Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight -(34) Project [codegen id : 5] -Output [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +(30) CometProject Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] +Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] + +(31) CometColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] ReadSchema: struct -(36) CometFilter +(33) CometFilter Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) -(37) CometProject +(34) CometProject Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(38) CometColumnarToRow [codegen id : 4] +(35) CometColumnarToRow [codegen id : 1] Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) BroadcastExchange +(36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(40) BroadcastHashJoin [codegen id : 5] +(37) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(38) Project [codegen id : 2] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(42) HashAggregate [codegen id : 5] +(39) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum#39] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(43) CometColumnarExchange +(40) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 6] +(41) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(45) HashAggregate [codegen id : 6] +(42) HashAggregate [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(46) HashAggregate [codegen id : 6] +(43) HashAggregate [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] Aggregate Attributes [2]: [sum#43, isEmpty#44] Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(47) CometColumnarExchange +(44) CometColumnarExchange Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(48) CometColumnarToRow [codegen id : 7] +(45) CometColumnarToRow [codegen id : 4] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(49) HashAggregate [codegen id : 7] +(46) HashAggregate [codegen id : 4] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(50) Filter [codegen id : 7] +(47) Filter [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) -(51) CometColumnarExchange +(48) CometColumnarExchange Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(52) CometSort +(49) CometSort Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(53) CometColumnarToRow [codegen id : 8] +(50) CometColumnarToRow [codegen id : 5] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== -Subquery:1 Hosting operator id = 50 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (84) -+- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * Project (76) - +- * BroadcastHashJoin Inner BuildRight (75) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * CometColumnarToRow (60) - : : : : +- CometProject (59) - : : : : +- CometSortMergeJoin (58) - : : : : :- CometSort (55) - : : : : : +- ReusedExchange (54) - : : : : +- CometSort (57) - : : : : +- ReusedExchange (56) - : : : +- ReusedExchange (61) - : : +- BroadcastExchange (68) - : : +- * CometColumnarToRow (67) - : : +- CometProject (66) - : : +- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (64) - : +- ReusedExchange (71) - +- ReusedExchange (74) - - -(54) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +* HashAggregate (80) ++- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- * CometColumnarToRow (69) + : +- CometProject (68) + : +- CometBroadcastHashJoin (67) + : :- CometProject (65) + : : +- CometBroadcastHashJoin (64) + : : :- CometProject (59) + : : : +- CometBroadcastHashJoin (58) + : : : :- CometProject (56) + : : : : +- CometSortMergeJoin (55) + : : : : :- CometSort (52) + : : : : : +- ReusedExchange (51) + : : : : +- CometSort (54) + : : : : +- ReusedExchange (53) + : : : +- ReusedExchange (57) + : : +- CometBroadcastExchange (63) + : : +- CometProject (62) + : : +- CometFilter (61) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) + : +- ReusedExchange (66) + +- ReusedExchange (70) + + +(51) ReusedExchange [Reuses operator id: 4] Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -(55) CometSort +(52) CometSort Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] -(56) ReusedExchange [Reuses operator id: 9] +(53) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#56, sr_ticket_number#57] -(57) CometSort +(54) CometSort Input [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] -(58) CometSortMergeJoin +(55) CometSortMergeJoin Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] Right output [2]: [sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner -(59) CometProject +(56) CometProject Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -(60) CometColumnarToRow [codegen id : 5] -Input [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(61) ReusedExchange [Reuses operator id: 18] +(57) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -(62) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#53] -Right keys [1]: [s_store_sk#58] -Join type: Inner -Join condition: None +(58) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] +Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] +Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight -(63) Project [codegen id : 5] -Output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] +(59) CometProject Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] +Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(65) CometFilter +(61) CometFilter Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Condition : isnotnull(i_item_sk#60) -(66) CometProject +(62) CometProject Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] -(67) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(68) BroadcastExchange +(63) CometBroadcastExchange Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(69) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#51] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None +(64) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] +Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight -(70) Project [codegen id : 5] -Output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +(65) CometProject Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -(71) ReusedExchange [Reuses operator id: 32] +(66) ReusedExchange [Reuses operator id: 28] Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(72) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#66] -Join type: Inner -Join condition: None +(67) CometBroadcastHashJoin +Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] +Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight -(73) Project [codegen id : 5] -Output [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +(68) CometProject Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] +Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] + +(69) CometColumnarToRow [codegen id : 2] +Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(74) ReusedExchange [Reuses operator id: 39] +(70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(75) BroadcastHashJoin [codegen id : 5] +(71) BroadcastHashJoin [codegen id : 2] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(76) Project [codegen id : 5] +(72) Project [codegen id : 2] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(77) HashAggregate [codegen id : 5] +(73) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum#71] Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(78) CometColumnarExchange +(74) CometColumnarExchange Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(79) CometColumnarToRow [codegen id : 6] +(75) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(80) HashAggregate [codegen id : 6] +(76) HashAggregate [codegen id : 3] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(81) HashAggregate [codegen id : 6] +(77) HashAggregate [codegen id : 3] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] Aggregate Attributes [2]: [sum#74, count#75] Results [2]: [sum#76, count#77] -(82) CometColumnarExchange +(78) CometColumnarExchange Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(83) CometColumnarToRow [codegen id : 7] +(79) CometColumnarToRow [codegen id : 4] Input [2]: [sum#76, count#77] -(84) HashAggregate [codegen id : 7] +(80) HashAggregate [codegen id : 4] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt index 0ed56d0b0d..7847aefd83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt @@ -13,37 +13,34 @@ CometColumnarToRow : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- 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 @@ -62,37 +59,34 @@ CometColumnarToRow +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- 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 @@ -102,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 54 out of 88 eligible operators (61%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index ce751cb78d..62f492f632 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -1,76 +1,70 @@ -WholeStageCodegen (8) +WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [paid] Subquery #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] ReusedExchange [sr_item_sk,sr_ticket_number] #5 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] @@ -83,33 +77,21 @@ WholeStageCodegen (8) CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (4) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt index 876b7c356b..397a3eb10c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt @@ -1,84 +1,75 @@ == Physical Plan == -* CometColumnarToRow (80) -+- CometTakeOrderedAndProject (79) - +- RowToColumnar (78) - +- Union (77) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * CometColumnarToRow (34) - : : : : : +- CometFilter (33) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (41) - : +- ReusedExchange (48) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * CometColumnarToRow (57) - : : : : +- CometFilter (56) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * CometColumnarToRow (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (67) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometUnion (69) + :- CometHashAggregate (29) + : +- CometExchange (28) + : +- CometHashAggregate (27) + : +- CometProject (26) + : +- CometBroadcastHashJoin (25) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- CometBroadcastExchange (24) + : +- CometProject (23) + : +- CometFilter (22) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + :- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (37) + : : : +- CometBroadcastHashJoin (36) + : : : :- CometProject (34) + : : : : +- CometBroadcastHashJoin (33) + : : : : :- CometFilter (31) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (32) + : : : +- ReusedExchange (35) + : : +- CometBroadcastExchange (41) + : : +- CometProject (40) + : : +- CometFilter (39) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) + : +- ReusedExchange (44) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + +- CometBroadcastExchange (63) + +- CometFilter (62) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -93,395 +84,354 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(11) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] +(17) CometProject +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#18] +Arguments: [s_store_sk#16, s_state#18] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#18] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] +(20) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] (21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] -Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(29) CometColumnarExchange -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(31) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] -Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] + +(24) CometBroadcastExchange +Input [2]: [i_item_sk#19, i_item_id#21] +Arguments: [i_item_sk#19, i_item_id#21] + +(25) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] +Right output [2]: [i_item_sk#19, i_item_id#21] +Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight + +(26) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] +Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] + +(27) CometHashAggregate +Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [2]: [i_item_id#22, s_state#23] +Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] + +(28) CometExchange +Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(29) CometHashAggregate +Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Keys [2]: [i_item_id#22, s_state#23] +Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] + +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(33) CometFilter -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) - -(34) CometColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +(31) CometFilter +Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) -(35) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#61] +(32) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#45] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#61] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Right output [1]: [cd_demo_sk#45] +Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight -(37) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#61] +(34) CometProject +Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] +Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -(38) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#62] +(35) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#46] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#62] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Right output [1]: [d_date_sk#46] +Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight -(40) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#62] +(37) CometProject +Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] +Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#63, s_state#64] +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#47, s_state#48] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [s_store_sk#63, s_state#64] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#64, 2, true, false, true) = TN) AND isnotnull(s_store_sk#63)) +(39) CometFilter +Input [2]: [s_store_sk#47, s_state#48] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) + +(40) CometProject +Input [2]: [s_store_sk#47, s_state#48] +Arguments: [s_store_sk#47], [s_store_sk#47] + +(41) CometBroadcastExchange +Input [1]: [s_store_sk#47] +Arguments: [s_store_sk#47] + +(42) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +Right output [1]: [s_store_sk#47] +Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight (43) CometProject -Input [2]: [s_store_sk#63, s_state#64] -Arguments: [s_store_sk#63], [s_store_sk#63] - -(44) CometColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#63] - -(45) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#63] - -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#65, i_item_id#20] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#20] - -(51) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#20, agg1#66, agg2#67, agg3#68, agg4#69] -Keys [1]: [i_item_id#20] -Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] -Aggregate Attributes [8]: [sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] -Results [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(52) CometColumnarExchange -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(53) CometColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Keys [1]: [i_item_id#20] -Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] -Aggregate Attributes [4]: [avg(agg1#66)#86, avg(UnscaledValue(agg2#67))#87, avg(UnscaledValue(agg3#68))#88, avg(UnscaledValue(agg4#69))#89] -Results [7]: [i_item_id#20, null AS s_state#90, 1 AS g_state#91, avg(agg1#66)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#67))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#68))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#69))#89 / 100.0) as decimal(11,6)) AS agg4#95] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] +Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] + +(44) ReusedExchange [Reuses operator id: 24] +Output [2]: [i_item_sk#49, i_item_id#21] + +(45) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +Right output [2]: [i_item_sk#49, i_item_id#21] +Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight + +(46) CometProject +Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] +Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] + +(47) CometHashAggregate +Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] +Keys [1]: [i_item_id#21] +Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] + +(48) CometExchange +Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(49) CometHashAggregate +Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Keys [1]: [i_item_id#21] +Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] + +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] +PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) CometFilter -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) - -(57) CometColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +(51) CometFilter +Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) -(58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#105] +(52) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#71] -(59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#105] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Right output [1]: [cd_demo_sk#71] +Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight -(60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#105] +(54) CometProject +Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] +Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -(61) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#106] +(55) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#72] -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#106] -Join type: Inner -Join condition: None +(56) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Right output [1]: [d_date_sk#72] +Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] +(57) CometProject +Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] +Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -(64) ReusedExchange [Reuses operator id: 45] -Output [1]: [s_store_sk#107] +(58) ReusedExchange [Reuses operator id: 41] +Output [1]: [s_store_sk#73] -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#107] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +Right output [1]: [s_store_sk#73] +Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#107] +(60) CometProject +Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] +Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#108] +(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [i_item_sk#108] -Condition : isnotnull(i_item_sk#108) +(62) CometFilter +Input [1]: [i_item_sk#74] +Condition : isnotnull(i_item_sk#74) -(69) CometColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#108] +(63) CometBroadcastExchange +Input [1]: [i_item_sk#74] +Arguments: [i_item_sk#74] -(70) BroadcastExchange -Input [1]: [i_item_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(64) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +Right output [1]: [i_item_sk#74] +Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#108] -Join type: Inner -Join condition: None +(65) CometProject +Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] +Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] -(72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#109, ss_list_price#100 AS agg2#110, ss_coupon_amt#102 AS agg3#111, ss_sales_price#101 AS agg4#112] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#108] - -(73) HashAggregate [codegen id : 17] -Input [4]: [agg1#109, agg2#110, agg3#111, agg4#112] +(66) CometHashAggregate +Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] Keys: [] -Functions [4]: [partial_avg(agg1#109), partial_avg(UnscaledValue(agg2#110)), partial_avg(UnscaledValue(agg3#111)), partial_avg(UnscaledValue(agg4#112))] -Aggregate Attributes [8]: [sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -Results [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] - -(74) CometColumnarExchange -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] -(75) CometColumnarToRow [codegen id : 18] -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +(67) CometExchange +Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(76) HashAggregate [codegen id : 18] -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +(68) CometHashAggregate +Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] Keys: [] -Functions [4]: [avg(agg1#109), avg(UnscaledValue(agg2#110)), avg(UnscaledValue(agg3#111)), avg(UnscaledValue(agg4#112))] -Aggregate Attributes [4]: [avg(agg1#109)#129, avg(UnscaledValue(agg2#110))#130, avg(UnscaledValue(agg3#111))#131, avg(UnscaledValue(agg4#112))#132] -Results [7]: [null AS i_item_id#133, null AS s_state#134, 1 AS g_state#135, avg(agg1#109)#129 AS agg1#136, cast((avg(UnscaledValue(agg2#110))#130 / 100.0) as decimal(11,6)) AS agg2#137, cast((avg(UnscaledValue(agg3#111))#131 / 100.0) as decimal(11,6)) AS agg3#138, cast((avg(UnscaledValue(agg4#112))#132 / 100.0) as decimal(11,6)) AS agg4#139] - -(77) Union +Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] -(78) RowToColumnar -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(69) CometUnion +Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] +Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] -(79) CometTakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#21 ASC NULLS FIRST,s_state#22 ASC NULLS FIRST], output=[i_item_id#21,s_state#22,g_state#47,agg1#48,agg2#49,agg3#50,agg4#51]), [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51], 100, 0, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(70) CometTakeOrderedAndProject +Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -(80) CometColumnarToRow [codegen id : 19] -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(71) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometProject (83) - +- CometFilter (82) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#140] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter -Input [2]: [d_date_sk#14, d_year#140] -Condition : ((isnotnull(d_year#140) AND (d_year#140 = 1998)) AND isnotnull(d_date_sk#14)) +(73) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(83) CometProject -Input [2]: [d_date_sk#14, d_year#140] +(74) CometProject +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(85) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index c26488e3c8..affbc9ef37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -1,118 +1,99 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 48 out of 95 eligible operators (50%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt index 293a5fc379..0d18ca5626 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt @@ -1,126 +1,83 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - RowToColumnar - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange [i_item_id,s_state] #1 + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] + CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange [i_item_id] #7 + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [s_store_sk] #8 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange #9 + CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange [i_item_sk] #10 + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt index 876b7c356b..397a3eb10c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt @@ -1,84 +1,75 @@ == Physical Plan == -* CometColumnarToRow (80) -+- CometTakeOrderedAndProject (79) - +- RowToColumnar (78) - +- Union (77) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * CometColumnarToRow (34) - : : : : : +- CometFilter (33) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (41) - : +- ReusedExchange (48) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * CometColumnarToRow (57) - : : : : +- CometFilter (56) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * CometColumnarToRow (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (67) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometUnion (69) + :- CometHashAggregate (29) + : +- CometExchange (28) + : +- CometHashAggregate (27) + : +- CometProject (26) + : +- CometBroadcastHashJoin (25) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- CometBroadcastExchange (24) + : +- CometProject (23) + : +- CometFilter (22) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) + :- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (37) + : : : +- CometBroadcastHashJoin (36) + : : : :- CometProject (34) + : : : : +- CometBroadcastHashJoin (33) + : : : : :- CometFilter (31) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (32) + : : : +- ReusedExchange (35) + : : +- CometBroadcastExchange (41) + : : +- CometProject (40) + : : +- CometFilter (39) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) + : +- ReusedExchange (44) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + +- CometBroadcastExchange (63) + +- CometFilter (62) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -93,395 +84,354 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(11) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] +(17) CometProject +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#18] +Arguments: [s_store_sk#16, s_state#18] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#18] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] +(20) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] (21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] -Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(29) CometColumnarExchange -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(31) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] -Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] + +(24) CometBroadcastExchange +Input [2]: [i_item_sk#19, i_item_id#21] +Arguments: [i_item_sk#19, i_item_id#21] + +(25) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] +Right output [2]: [i_item_sk#19, i_item_id#21] +Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight + +(26) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] +Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] + +(27) CometHashAggregate +Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [2]: [i_item_id#22, s_state#23] +Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] + +(28) CometExchange +Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(29) CometHashAggregate +Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Keys [2]: [i_item_id#22, s_state#23] +Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] + +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(33) CometFilter -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) - -(34) CometColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] +(31) CometFilter +Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) -(35) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#61] +(32) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#45] -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#61] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Right output [1]: [cd_demo_sk#45] +Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight -(37) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#61] +(34) CometProject +Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] +Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -(38) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#62] +(35) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#46] -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#62] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] +Right output [1]: [d_date_sk#46] +Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight -(40) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#62] +(37) CometProject +Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] +Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#63, s_state#64] +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#47, s_state#48] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [s_store_sk#63, s_state#64] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#64, 2, true, false, true) = TN) AND isnotnull(s_store_sk#63)) +(39) CometFilter +Input [2]: [s_store_sk#47, s_state#48] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) + +(40) CometProject +Input [2]: [s_store_sk#47, s_state#48] +Arguments: [s_store_sk#47], [s_store_sk#47] + +(41) CometBroadcastExchange +Input [1]: [s_store_sk#47] +Arguments: [s_store_sk#47] + +(42) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +Right output [1]: [s_store_sk#47] +Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight (43) CometProject -Input [2]: [s_store_sk#63, s_state#64] -Arguments: [s_store_sk#63], [s_store_sk#63] - -(44) CometColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#63] - -(45) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#63] - -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#65, i_item_id#20] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#65] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#65, i_item_id#20] - -(51) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#20, agg1#66, agg2#67, agg3#68, agg4#69] -Keys [1]: [i_item_id#20] -Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] -Aggregate Attributes [8]: [sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] -Results [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(52) CometColumnarExchange -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(53) CometColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#20, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Keys [1]: [i_item_id#20] -Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] -Aggregate Attributes [4]: [avg(agg1#66)#86, avg(UnscaledValue(agg2#67))#87, avg(UnscaledValue(agg3#68))#88, avg(UnscaledValue(agg4#69))#89] -Results [7]: [i_item_id#20, null AS s_state#90, 1 AS g_state#91, avg(agg1#66)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#67))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#68))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#69))#89 / 100.0) as decimal(11,6)) AS agg4#95] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] +Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] + +(44) ReusedExchange [Reuses operator id: 24] +Output [2]: [i_item_sk#49, i_item_id#21] + +(45) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] +Right output [2]: [i_item_sk#49, i_item_id#21] +Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight + +(46) CometProject +Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] +Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] + +(47) CometHashAggregate +Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] +Keys [1]: [i_item_id#21] +Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] + +(48) CometExchange +Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(49) CometHashAggregate +Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Keys [1]: [i_item_id#21] +Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] + +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] +PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) CometFilter -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) - -(57) CometColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +(51) CometFilter +Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) -(58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#105] +(52) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#71] -(59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#105] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Right output [1]: [cd_demo_sk#71] +Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight -(60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#105] +(54) CometProject +Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] +Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -(61) ReusedExchange [Reuses operator id: 85] -Output [1]: [d_date_sk#106] +(55) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#72] -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#106] -Join type: Inner -Join condition: None +(56) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] +Right output [1]: [d_date_sk#72] +Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] +(57) CometProject +Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] +Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -(64) ReusedExchange [Reuses operator id: 45] -Output [1]: [s_store_sk#107] +(58) ReusedExchange [Reuses operator id: 41] +Output [1]: [s_store_sk#73] -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#107] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +Right output [1]: [s_store_sk#73] +Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#107] +(60) CometProject +Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] +Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#108] +(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [1]: [i_item_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(68) CometFilter -Input [1]: [i_item_sk#108] -Condition : isnotnull(i_item_sk#108) +(62) CometFilter +Input [1]: [i_item_sk#74] +Condition : isnotnull(i_item_sk#74) -(69) CometColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#108] +(63) CometBroadcastExchange +Input [1]: [i_item_sk#74] +Arguments: [i_item_sk#74] -(70) BroadcastExchange -Input [1]: [i_item_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(64) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] +Right output [1]: [i_item_sk#74] +Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight -(71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#108] -Join type: Inner -Join condition: None +(65) CometProject +Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] +Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] -(72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#109, ss_list_price#100 AS agg2#110, ss_coupon_amt#102 AS agg3#111, ss_sales_price#101 AS agg4#112] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#108] - -(73) HashAggregate [codegen id : 17] -Input [4]: [agg1#109, agg2#110, agg3#111, agg4#112] +(66) CometHashAggregate +Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] Keys: [] -Functions [4]: [partial_avg(agg1#109), partial_avg(UnscaledValue(agg2#110)), partial_avg(UnscaledValue(agg3#111)), partial_avg(UnscaledValue(agg4#112))] -Aggregate Attributes [8]: [sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -Results [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] - -(74) CometColumnarExchange -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] -(75) CometColumnarToRow [codegen id : 18] -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +(67) CometExchange +Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(76) HashAggregate [codegen id : 18] -Input [8]: [sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128] +(68) CometHashAggregate +Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] Keys: [] -Functions [4]: [avg(agg1#109), avg(UnscaledValue(agg2#110)), avg(UnscaledValue(agg3#111)), avg(UnscaledValue(agg4#112))] -Aggregate Attributes [4]: [avg(agg1#109)#129, avg(UnscaledValue(agg2#110))#130, avg(UnscaledValue(agg3#111))#131, avg(UnscaledValue(agg4#112))#132] -Results [7]: [null AS i_item_id#133, null AS s_state#134, 1 AS g_state#135, avg(agg1#109)#129 AS agg1#136, cast((avg(UnscaledValue(agg2#110))#130 / 100.0) as decimal(11,6)) AS agg2#137, cast((avg(UnscaledValue(agg3#111))#131 / 100.0) as decimal(11,6)) AS agg3#138, cast((avg(UnscaledValue(agg4#112))#132 / 100.0) as decimal(11,6)) AS agg4#139] - -(77) Union +Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] -(78) RowToColumnar -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(69) CometUnion +Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] +Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] -(79) CometTakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#21 ASC NULLS FIRST,s_state#22 ASC NULLS FIRST], output=[i_item_id#21,s_state#22,g_state#47,agg1#48,agg2#49,agg3#50,agg4#51]), [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51], 100, 0, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(70) CometTakeOrderedAndProject +Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -(80) CometColumnarToRow [codegen id : 19] -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +(71) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometProject (83) - +- CometFilter (82) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#140] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter -Input [2]: [d_date_sk#14, d_year#140] -Condition : ((isnotnull(d_year#140) AND (d_year#140 = 1998)) AND isnotnull(d_date_sk#14)) +(73) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(83) CometProject -Input [2]: [d_date_sk#14, d_year#140] +(74) CometProject +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(85) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt index c26488e3c8..affbc9ef37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt @@ -1,118 +1,99 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 48 out of 95 eligible operators (50%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 293a5fc379..0d18ca5626 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt @@ -1,126 +1,83 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - RowToColumnar - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange [i_item_id,s_state] #1 + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] + CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [cd_demo_sk] #3 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_state] #5 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange [i_item_id] #7 + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [s_store_sk] #8 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometExchange #9 + CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk] + CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange [i_item_sk] #10 + CometFilter [i_item_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt index 0447825143..50af5f4f75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt @@ -1,39 +1,37 @@ == Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (26) +* CometColumnarToRow (33) ++- CometSort (32) + +- CometExchange (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) + +- CometBroadcastExchange (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -48,184 +46,177 @@ ReadSchema: struct + +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(23) CometHashAggregate +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(24) CometFilter Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(27) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(28) CometProject +(27) CometProject Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange +(28) CometBroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] +(30) CometProject Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(33) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(34) CometSort +(32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] -(35) CometColumnarToRow [codegen id : 7] +(33) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) +BroadcastExchange (38) ++- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index 7cb35600f0..3d9d23d2cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 20 out of 37 eligible operators (54%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt index 568e86c9fe..d57afd4fcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt @@ -1,59 +1,43 @@ -WholeStageCodegen (7) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] + CometExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt index 0447825143..50af5f4f75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt @@ -1,39 +1,37 @@ == Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (26) +* CometColumnarToRow (33) ++- CometSort (32) + +- CometExchange (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometFilter (24) + : +- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) + +- CometBroadcastExchange (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -48,184 +46,177 @@ ReadSchema: struct + +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(23) CometHashAggregate +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(24) CometFilter Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(27) CometFilter +(26) CometFilter Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Condition : isnotnull(c_customer_sk#18) -(28) CometProject +(27) CometProject Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange +(28) CometBroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] +(30) CometProject Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] +Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(33) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(34) CometSort +(32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] -(35) CometColumnarToRow [codegen id : 7] +(33) CometColumnarToRow [codegen id : 1] Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) +BroadcastExchange (38) ++- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(40) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt index 7cb35600f0..3d9d23d2cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 20 out of 37 eligible operators (54%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index 568e86c9fe..d57afd4fcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -1,59 +1,43 @@ -WholeStageCodegen (7) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [ss_ticket_number,ss_customer_sk,cnt] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] + CometExchange [ss_ticket_number,ss_customer_sk] #2 + CometHashAggregate [ss_ticket_number,ss_customer_sk,count] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk] + CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange [s_store_sk] #5 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange [hd_demo_sk] #6 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt index f08a74ef15..b43ed9b192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt @@ -1,51 +1,51 @@ == Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * CometColumnarToRow (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * CometColumnarToRow (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (34) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * CometColumnarToRow (45) + +- CometColumnarExchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * CometColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * CometColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (40) + +- * CometColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,214 +59,217 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(12) CometColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) CometColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) CometColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] +(31) CometProject +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#20] +(32) CometColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#22] -(31) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] +(35) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#22] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(35) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) +(37) CometFilter +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) -(36) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(38) CometProject +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(39) CometColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(38) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(40) BroadcastExchange +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(39) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] +Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(42) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(42) Project [codegen id : 5] +Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(43) HashAggregate [codegen id : 5] +Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] +(44) CometColumnarExchange +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) RowToColumnar -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(45) CometColumnarToRow [codegen id : 6] +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) CometTakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(46) HashAggregate [codegen id : 6] +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] +Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] -(47) CometColumnarToRow [codegen id : 11] -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(47) TakeOrderedAndProject +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) +- * CometColumnarToRow (51) +- CometProject (50) @@ -275,18 +278,18 @@ BroadcastExchange (52) (48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] -Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 2002)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) (50) CometProject -Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] (51) CometColumnarToRow [codegen id : 1] @@ -294,10 +297,10 @@ Input [1]: [d_date_sk#9] (52) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index 8978af8750..be2d144fe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -1,69 +1,63 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + TakeOrderedAndProject [COMET: ] ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 25 out of 54 eligible operators (46%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt index 9e3618d14c..fea6a32741 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt @@ -1,81 +1,73 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + WholeStageCodegen (6) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (5) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt index f08a74ef15..b43ed9b192 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt @@ -1,51 +1,51 @@ == Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- RowToColumnar (45) - +- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * CometColumnarToRow (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * CometColumnarToRow (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (34) +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- * CometColumnarToRow (45) + +- CometColumnarExchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * CometColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * CometColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (40) + +- * CometColumnarToRow (39) + +- CometProject (38) + +- CometFilter (37) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -59,214 +59,217 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(12) CometColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) CometColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) CometColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] +(31) CometProject +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#20] +(32) CometColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#22] -(31) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] +(35) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#22] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(35) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) +(37) CometFilter +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) -(36) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(38) CometProject +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(39) CometColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(38) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(40) BroadcastExchange +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(39) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] +Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(42) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(42) Project [codegen id : 5] +Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(43) HashAggregate [codegen id : 5] +Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] +(44) CometColumnarExchange +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) RowToColumnar -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(45) CometColumnarToRow [codegen id : 6] +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) CometTakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(46) HashAggregate [codegen id : 6] +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] +Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] -(47) CometColumnarToRow [codegen id : 11] -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(47) TakeOrderedAndProject +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) +- * CometColumnarToRow (51) +- CometProject (50) @@ -275,18 +278,18 @@ BroadcastExchange (52) (48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct (49) CometFilter -Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] -Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 2002)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#9)) +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) (50) CometProject -Input [3]: [d_date_sk#9, d_year#77, d_qoy#78] +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] (51) CometColumnarToRow [codegen id : 1] @@ -294,10 +297,10 @@ Input [1]: [d_date_sk#9] (52) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt index 8978af8750..be2d144fe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt @@ -1,69 +1,63 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + TakeOrderedAndProject [COMET: ] ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- 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 + : : : : : +- CometColumnarToRow + : : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 25 out of 54 eligible operators (46%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index 9e3618d14c..fea6a32741 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -1,81 +1,73 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + WholeStageCodegen (6) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (5) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + CometColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt index 0c6a3e9fb9..e22a5c492c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt @@ -1,49 +1,44 @@ == Physical Plan == -* CometColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- RowToColumnar (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * CometColumnarToRow (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * CometColumnarToRow (17) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -57,233 +52,216 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customsk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customsk#14] +Child 1 Input [1]: [customsk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customsk#14] +Arguments: [customsk#14] -(22) BroadcastExchange -Input [1]: [customsk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customsk#14] +Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(27) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(28) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#20] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(29) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(27) CometBroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#22] +Arguments: [ca_address_sk#20, ca_state#22] -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [2]: [ca_address_sk#20, ca_state#22] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(31) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#20] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#20] +(29) CometProject +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] +Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) +(31) CometFilter +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) -(34) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(32) CometProject +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(35) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(33) CometBroadcastExchange +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(34) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, ca_state#22] +Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None +(35) CometProject +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(38) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(36) CometHashAggregate +Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(37) CometExchange +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(38) CometHashAggregate +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -(41) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(39) CometTakeOrderedAndProject +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -(42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] - -(43) RowToColumnar -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -(44) CometTakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -(45) CometColumnarToRow [codegen id : 11] -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(40) CometColumnarToRow [codegen id : 1] +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#77, d_qoy#78] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] -Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 1999)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#7)) +(42) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(48) CometProject -Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] +(43) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(49) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(50) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index fbfef118e0..2cdc75e15e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -1,67 +1,56 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt index 6614ba7532..69e8d4868c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt @@ -1,78 +1,52 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customsk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt index 0c6a3e9fb9..e22a5c492c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt @@ -1,49 +1,44 @@ == Physical Plan == -* CometColumnarToRow (45) -+- CometTakeOrderedAndProject (44) - +- RowToColumnar (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * CometColumnarToRow (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * CometColumnarToRow (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * CometColumnarToRow (17) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (32) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -57,233 +52,216 @@ ReadSchema: struct -(5) CometColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) CometColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) CometColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customsk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customsk#14] +Child 1 Input [1]: [customsk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customsk#14] +Arguments: [customsk#14] -(22) BroadcastExchange -Input [1]: [customsk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customsk#14] +Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(27) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(28) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#20] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(29) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(27) CometBroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#22] +Arguments: [ca_address_sk#20, ca_state#22] -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [2]: [ca_address_sk#20, ca_state#22] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(31) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#20] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#20] +(29) CometProject +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] +Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) +(31) CometFilter +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) -(34) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(32) CometProject +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(35) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(33) CometBroadcastExchange +Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(34) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, ca_state#22] +Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None +(35) CometProject +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(38) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] +(36) CometHashAggregate +Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(37) CometExchange +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(38) CometHashAggregate +Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -(41) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +(39) CometTakeOrderedAndProject +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -(42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] - -(43) RowToColumnar -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -(44) CometTakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#20 ASC NULLS FIRST,cd_gender#27 ASC NULLS FIRST,cd_marital_status#28 ASC NULLS FIRST,cd_dep_count#24 ASC NULLS FIRST,cd_dep_employed_count#25 ASC NULLS FIRST,cd_dep_college_count#26 ASC NULLS FIRST], output=[ca_state#20,cd_gender#27,cd_marital_status#28,cd_dep_count#24,cnt1#65,avg(cd_dep_count)#66,max(cd_dep_count)#67,sum(cd_dep_count)#68,cd_dep_employed_count#25,cnt2#69,avg(cd_dep_employed_count)#70,max(cd_dep_employed_count)#71,sum(cd_dep_employed_count)#72,cd_dep_college_count#26,cnt3#73,avg(cd_dep_college_count)#74,max(cd_dep_college_count)#75,sum(cd_dep_college_count)#76]), [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76], 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -(45) CometColumnarToRow [codegen id : 11] -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +(40) CometColumnarToRow [codegen id : 1] +Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#77, d_qoy#78] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] -Condition : ((((isnotnull(d_year#77) AND isnotnull(d_qoy#78)) AND (d_year#77 = 1999)) AND (d_qoy#78 < 4)) AND isnotnull(d_date_sk#7)) +(42) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(48) CometProject -Input [3]: [d_date_sk#7, d_year#77, d_qoy#78] +(43) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(49) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(50) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt index fbfef118e0..2cdc75e15e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt @@ -1,67 +1,56 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 25 out of 52 eligible operators (48%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt index 6614ba7532..69e8d4868c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt @@ -1,78 +1,52 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - RowToColumnar - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastExchange [ss_customer_sk] #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange [customsk] #5 + CometUnion [customsk] + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customsk] + CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ca_address_sk,ca_state] #6 + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt index 68c5730e98..2562693a01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt @@ -1,54 +1,47 @@ == Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- RowToColumnar (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- ReusedExchange (32) +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * CometColumnarToRow (40) + +- CometSort (39) + +- CometExchange (38) + +- CometHashAggregate (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometUnion (34) + :- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + +- CometHashAggregate (33) + +- CometExchange (32) + +- CometHashAggregate (31) + +- CometHashAggregate (30) + +- ReusedExchange (29) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -63,252 +56,220 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] +(11) CometProject +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] +(12) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#12, i_category#13] +Arguments: [i_item_sk#9, i_class#12, i_category#13] -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#12, i_category#13] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(14) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] +(17) CometProject +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14], [s_store_sk#14] -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [s_store_sk#14] +Arguments: [s_store_sk#14] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +Right output [1]: [s_store_sk#14] +Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] +(20) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -(21) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Keys [2]: [i_category#12, i_class#11] +(21) CometHashAggregate +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +Keys [2]: [i_category#13, i_class#12] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [i_category#12, i_class#11, sum#17, sum#18] -(22) CometColumnarExchange -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [4]: [i_category#13, i_class#12, sum#16, sum#17] +Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(24) HashAggregate [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Keys [2]: [i_category#12, i_class#11] +(23) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#16, sum#17] +Keys [2]: [i_category#13, i_class#12] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(26) CometColumnarToRow [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(27) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] - -(28) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] -Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] -Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(29) CometColumnarExchange -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] -Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] -Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(33) CometColumnarToRow [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(34) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] - -(35) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] -Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(36) CometColumnarExchange -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#13, i_class#12, sum#18, sum#19] + +(25) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#18, sum#19] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] + +(26) CometHashAggregate +Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] +Keys [1]: [i_category#13] +Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] + +(27) CometExchange +Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] +Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +(28) CometHashAggregate +Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] +Keys [1]: [i_category#13] +Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] -(38) HashAggregate [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#13, i_class#12, sum#28, sum#29] + +(30) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#28, sum#29] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] + +(31) CometHashAggregate +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] -Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] -Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as decimal(38,20)) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -(39) Union +(32) CometExchange +Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +(33) CometHashAggregate +Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Keys: [] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -(41) CometColumnarExchange -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(34) CometUnion +Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] +Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] -(42) CometHashAggregate -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +(35) CometHashAggregate +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] Functions: [] -(43) CometExchange -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(36) CometExchange +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(44) CometSort -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] +(37) CometHashAggregate +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Functions: [] -(45) CometColumnarToRow [codegen id : 19] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +(38) CometExchange +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(46) Window -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] +(39) CometSort +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] -(47) Project [codegen id : 20] -Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] +(40) CometColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -(48) RowToColumnar -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(41) Window +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] -(49) CometTakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#26 DESC NULLS LAST,CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST,rank_within_parent#73 ASC NULLS FIRST], output=[gross_margin#21,i_category#22,i_class#23,lochierarchy#26,rank_within_parent#73]), [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73], 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] -(50) CometColumnarToRow [codegen id : 21] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(43) TakeOrderedAndProject +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#74] +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter -Input [2]: [d_date_sk#7, d_year#74] -Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) +(45) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(53) CometProject -Input [2]: [d_date_sk#7, d_year#74] +(46) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(54) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(55) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index c50592042c..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -1,124 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 52 out of 99 eligible operators (52%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt index 1501a1fbd4..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt @@ -1,85 +1,55 @@ -WholeStageCodegen (21) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (20) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt index 68c5730e98..2562693a01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt @@ -1,54 +1,47 @@ == Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- RowToColumnar (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- ReusedExchange (32) +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * CometColumnarToRow (40) + +- CometSort (39) + +- CometExchange (38) + +- CometHashAggregate (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometUnion (34) + :- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + +- CometHashAggregate (33) + +- CometExchange (32) + +- CometHashAggregate (31) + +- CometHashAggregate (30) + +- ReusedExchange (29) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -63,252 +56,220 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] +(11) CometProject +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] +(12) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#12, i_category#13] +Arguments: [i_item_sk#9, i_class#12, i_category#13] -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#12, i_category#13] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(14) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] +(17) CometProject +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14], [s_store_sk#14] -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [s_store_sk#14] +Arguments: [s_store_sk#14] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +Right output [1]: [s_store_sk#14] +Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] +(20) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -(21) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Keys [2]: [i_category#12, i_class#11] +(21) CometHashAggregate +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] +Keys [2]: [i_category#13, i_class#12] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [i_category#12, i_class#11, sum#17, sum#18] -(22) CometColumnarExchange -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [4]: [i_category#13, i_class#12, sum#16, sum#17] +Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(24) HashAggregate [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Keys [2]: [i_category#12, i_class#11] +(23) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#16, sum#17] +Keys [2]: [i_category#13, i_class#12] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(26) CometColumnarToRow [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(27) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] - -(28) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] -Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] -Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(29) CometColumnarExchange -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] -Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] -Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(33) CometColumnarToRow [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(34) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] - -(35) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] -Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(36) CometColumnarExchange -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#13, i_class#12, sum#18, sum#19] + +(25) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#18, sum#19] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] + +(26) CometHashAggregate +Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] +Keys [1]: [i_category#13] +Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] + +(27) CometExchange +Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] +Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +(28) CometHashAggregate +Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] +Keys [1]: [i_category#13] +Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] -(38) HashAggregate [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#13, i_class#12, sum#28, sum#29] + +(30) CometHashAggregate +Input [4]: [i_category#13, i_class#12, sum#28, sum#29] +Keys [2]: [i_category#13, i_class#12] +Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] + +(31) CometHashAggregate +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] -Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] -Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as decimal(38,20)) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -(39) Union +(32) CometExchange +Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +(33) CometHashAggregate +Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Keys: [] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -(41) CometColumnarExchange -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(34) CometUnion +Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] +Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] -(42) CometHashAggregate -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] +(35) CometHashAggregate +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] Functions: [] -(43) CometExchange -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(36) CometExchange +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(44) CometSort -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] +(37) CometHashAggregate +Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] +Functions: [] -(45) CometColumnarToRow [codegen id : 19] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] +(38) CometExchange +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(46) Window -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] +(39) CometSort +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] -(47) Project [codegen id : 20] -Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] +(40) CometColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -(48) RowToColumnar -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(41) Window +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] +Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] -(49) CometTakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#26 DESC NULLS LAST,CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST,rank_within_parent#73 ASC NULLS FIRST], output=[gross_margin#21,i_category#22,i_class#23,lochierarchy#26,rank_within_parent#73]), [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73], 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] -(50) CometColumnarToRow [codegen id : 21] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] +(43) TakeOrderedAndProject +Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] +Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#74] +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter -Input [2]: [d_date_sk#7, d_year#74] -Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) +(45) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(53) CometProject -Input [2]: [d_date_sk#7, d_year#74] +(46) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(54) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(55) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt index c50592042c..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt @@ -1,124 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 52 out of 99 eligible operators (52%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt index 1501a1fbd4..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt @@ -1,85 +1,55 @@ -WholeStageCodegen (21) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (20) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt index 0c7c2a3bce..7b1f860f32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt @@ -1,57 +1,52 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometTakeOrderedAndProject (52) - +- RowToColumnar (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -69,10 +64,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -80,39 +72,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] +Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +(8) CometProject Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(11) ReusedExchange [Reuses operator id: 57] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(10) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) + +(11) CometBroadcastExchange +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [d_date_sk#11, d_year#12, d_moy#13] + +(12) CometBroadcastHashJoin +Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] +(13) CometProject Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] +Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] @@ -125,188 +124,166 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] +Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] +(18) CometProject Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +(20) CometExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +(21) CometHashAggregate +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(22) CometExchange +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(23) CometSort +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +(24) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] +(26) Filter [codegen id : 2] +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) +(27) Window +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] +(28) Filter [codegen id : 7] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] +(31) CometHashAggregate +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] +Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(32) CometExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +(37) BroadcastExchange +Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) RowToColumnar -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -(52) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#23,sum_sales#20,psum#42,nsum#43]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43], 100, 0, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +(47) Project [codegen id : 7] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(53) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +(48) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometFilter (55) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(56) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(57) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index d48e51fba0..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -1,122 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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 + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt index 329fe73435..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt @@ -1,91 +1,71 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - RowToColumnar - WholeStageCodegen (22) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (7) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt index 0c7c2a3bce..7b1f860f32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt @@ -1,57 +1,52 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometTakeOrderedAndProject (52) - +- RowToColumnar (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -69,10 +64,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -80,39 +72,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] +Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +(8) CometProject Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -(11) ReusedExchange [Reuses operator id: 57] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(10) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) + +(11) CometBroadcastExchange +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [d_date_sk#11, d_year#12, d_moy#13] + +(12) CometBroadcastHashJoin +Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] +Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] +(13) CometProject Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] +Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] @@ -125,188 +124,166 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] +Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] +(18) CometProject Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] +Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +(20) CometExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] +(21) CometHashAggregate +Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(22) CometExchange +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(23) CometSort +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] +(24) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] +(26) Filter [codegen id : 2] +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) +(27) Window +Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] +(28) Filter [codegen id : 7] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] +(31) CometHashAggregate +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] +Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(32) CometExchange +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +(37) BroadcastExchange +Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) RowToColumnar -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -(52) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#23,sum_sales#20,psum#42,nsum#43]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43], 100, 0, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +(47) Project [codegen id : 7] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(53) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] +(48) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometFilter (55) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(56) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(57) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt index d48e51fba0..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt @@ -1,122 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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 + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index 329fe73435..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt @@ -1,91 +1,71 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - RowToColumnar - WholeStageCodegen (22) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (7) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt index 9b6b457306..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -12,77 +12,71 @@ : +- Window (22) : +- * CometColumnarToRow (21) : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * CometColumnarToRow (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * CometColumnarToRow (56) - : : +- CometProject (55) - : : +- CometFilter (54) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -101,397 +95,368 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) CometColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] + +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight + +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(15) HashAggregate [codegen id : 3] +(16) CometHashAggregate Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(17) CometExchange +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(18) CometHashAggregate +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(19) CometExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(21) CometColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct (28) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (29) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometColumnarToRow [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(33) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(34) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#44, cr_item_sk#43] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#48] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(45) CometColumnarExchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] - -(48) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(32) CometFilter +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) + +(33) CometProject +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#40] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] + +(39) CometHashAggregate +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] + +(40) CometExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] + +(42) CometExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] + +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 + +(47) Window +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] + +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) + +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] + +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) - -(55) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(51) CometFilter +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(56) CometColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(57) BroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(59) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(60) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] -Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#83] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(71) CometColumnarExchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(74) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(55) CometFilter +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) + +(56) CometProject +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#65] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] + +(63) CometExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] + +(65) CometExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] + +(67) CometColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] + +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 + +(70) Window +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] + +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) + +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] + +(73) Union + +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(75) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(78) CometColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(80) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(87) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(81) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(89) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index 1a76c9c7f9..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -11,35 +11,31 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 @@ -47,30 +43,26 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- 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 @@ -78,29 +70,25 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 42 out of 87 eligible operators (48%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt index d836b433a6..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt @@ -1,140 +1,110 @@ -WholeStageCodegen (23) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) + WholeStageCodegen (10) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (3) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (6) + WholeStageCodegen (2) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (5) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (13) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (12) + WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (20) + WholeStageCodegen (8) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (19) + WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 9b6b457306..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) :- * Project (26) : +- * Filter (25) : +- Window (24) @@ -12,77 +12,71 @@ : +- Window (22) : +- * CometColumnarToRow (21) : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * CometColumnarToRow (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * CometColumnarToRow (56) - : : +- CometProject (55) - : : +- CometFilter (54) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -101,397 +95,368 @@ Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(4) CometColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) -(8) CometProject +(7) CometProject Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] + +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight + +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(15) HashAggregate [codegen id : 3] +(16) CometHashAggregate Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(17) CometExchange +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(18) CometHashAggregate +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(19) CometExchange +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(21) CometColumnarToRow [codegen id : 1] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 2] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 (24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 3] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct (28) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) (29) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometColumnarToRow [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(33) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(34) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#44, cr_item_sk#43] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#48] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(45) CometColumnarExchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [item#65, return_ratio#66, currency_ratio#67], [return_ratio#66 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] - -(48) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(32) CometFilter +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) + +(33) CometProject +Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] +Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] +Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#40] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] +Right output [1]: [d_date_sk#40] +Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] +Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] + +(39) CometHashAggregate +Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] +Keys [1]: [cs_item_sk#28] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] + +(40) CometExchange +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(41) CometHashAggregate +Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] +Keys [1]: [cs_item_sk#28] +Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] + +(42) CometExchange +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(43) CometSort +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window +Input [3]: [item#47, return_ratio#48, currency_ratio#49] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] + +(46) Sort [codegen id : 5] +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 + +(47) Window +Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] + +(48) Filter [codegen id : 6] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) + +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] + +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) - -(55) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(51) CometFilter +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(56) CometColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(57) BroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(59) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(60) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#72, ss_item_sk#71] -Right keys [2]: [sr_ticket_number#79, sr_item_sk#78] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#83] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(71) CometColumnarExchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [item#100, return_ratio#101, currency_ratio#102], [return_ratio#101 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] - -(74) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(55) CometFilter +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) + +(56) CometProject +Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] +Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] +Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#65] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] +Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] +Keys [1]: [ss_item_sk#53] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] + +(63) CometExchange +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(64) CometHashAggregate +Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Keys [1]: [ss_item_sk#53] +Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] + +(65) CometExchange +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(66) CometSort +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] + +(67) CometColumnarToRow [codegen id : 7] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] + +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 + +(70) Window +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] + +(71) Filter [codegen id : 9] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) + +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] + +(73) Union + +(74) HashAggregate [codegen id : 10] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(75) CometColumnarExchange +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) CometHashAggregate +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) CometTakeOrderedAndProject +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(78) CometColumnarToRow [codegen id : 11] +Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(80) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(87) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(81) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(89) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt index 1a76c9c7f9..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt @@ -11,35 +11,31 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 @@ -47,30 +43,26 @@ CometColumnarToRow : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- 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 @@ -78,29 +70,25 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 42 out of 87 eligible operators (48%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index d836b433a6..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt @@ -1,140 +1,110 @@ -WholeStageCodegen (23) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) + WholeStageCodegen (10) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (3) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (6) + WholeStageCodegen (2) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (5) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (13) + WholeStageCodegen (5) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (12) + WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (20) + WholeStageCodegen (8) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (19) + WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt index 204830d57b..f8a95a6db8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt @@ -1,83 +1,80 @@ == Physical Plan == -* CometColumnarToRow (79) -+- CometTakeOrderedAndProject (78) - +- RowToColumnar (77) - +- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * CometColumnarToRow (64) - : +- CometSort (63) - : +- CometExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) - : : +- CometColumnarExchange (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * CometColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) - : +- CometColumnarExchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * CometColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometColumnarExchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * CometColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * CometColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -92,372 +89,360 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(10) CometExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) HashAggregate [codegen id : 3] +(11) CometHashAggregate Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(12) CometExchange +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(12) CometSort -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(13) CometSort +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +(14) CometColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -(14) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(17) CometSort -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] +(18) CometSort +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(18) CometColumnarToRow [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(19) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] +(20) Window +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(20) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +(22) BroadcastExchange +Input [3]: [item_sk#15, sumws#16, rk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] Join type: Inner -Join condition: (rk#12 >= rk#15) - -(23) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] - -(24) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(25) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] - -(28) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Join condition: (rk#11 >= rk#14) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(28) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] + +(29) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(32) CometColumnarToRow [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(33) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#28, d_date#29] +(32) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) -(34) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(35) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +(34) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(36) HashAggregate [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [ss_item_sk#24, d_date#29, sum#31] +(35) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] -(37) CometColumnarExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(36) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -(38) CometColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] +(37) CometExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(39) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] -Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -(40) CometColumnarExchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(39) CometExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(41) CometSort -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] +(40) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(43) Window -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] +(42) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] -(44) Project [codegen id : 21] -Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] -Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] +(43) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(46) CometSort -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(45) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 19] -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(48) Window -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] +(47) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] -(49) Project [codegen id : 20] -Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] -Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] +(48) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] -(50) BroadcastExchange -Input [3]: [item_sk#39, sumss#40, rk#38] +(49) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [item_sk#33] -Right keys [1]: [item_sk#39] +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] Join type: Inner -Join condition: (rk#35 >= rk#38) +Join condition: (rk#32 >= rk#35) -(52) Project [codegen id : 21] -Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] -(53) HashAggregate [codegen id : 21] -Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [partial_sum(sumss#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(54) CometColumnarExchange -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(53) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometColumnarToRow [codegen id : 22] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(56) HashAggregate [codegen id : 22] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [sum(sumss#40)] -Aggregate Attributes [1]: [sum(sumss#40)#45] -Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] +(55) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] -(57) CometColumnarExchange -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(56) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(58) CometSort -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] +(57) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(59) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter +(58) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter -(60) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) +(59) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) -(61) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] +(60) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] -(62) CometExchange -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(63) CometSort -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] +(62) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] -(64) CometColumnarToRow [codegen id : 23] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(65) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(64) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(67) CometSort -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] -(68) CometColumnarToRow [codegen id : 46] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(69) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(70) Project [codegen id : 47] -Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] -Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] +(69) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] -(71) BroadcastExchange -Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] +(70) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 48] -Left keys [1]: [item_sk#47] -Right keys [1]: [item_sk#53] +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] Join type: Inner -Join condition: (rk#51 >= rk#52) - -(73) Project [codegen id : 48] -Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] - -(74) HashAggregate [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] - -(75) HashAggregate [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [max(web_sales#54), max(store_sales#55)] -Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] - -(76) Filter [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(77) RowToColumnar -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] - -(78) CometTakeOrderedAndProject -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#47 ASC NULLS FIRST,d_date#48 ASC NULLS FIRST], output=[item_sk#47,d_date#48,web_sales#49,store_sales#50,web_cumulative#62,store_cumulative#63]), [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63], 100, 0, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] - -(79) CometColumnarToRow [codegen id : 49] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Join condition: (rk#48 >= rk#49) + +(72) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(73) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometProject (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(80) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) +(78) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(82) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(79) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(83) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(84) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index 898288d55b..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -1,242 +1,216 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Window expressions are not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometSortMergeJoin + :- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 99 out of 196 eligible operators (50%). Final plan contains 44 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt index 91b14603ab..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt @@ -1,131 +1,116 @@ -WholeStageCodegen (49) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - RowToColumnar - WholeStageCodegen (48) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (24) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (4) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [item_sk,sumws,rk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (6) + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Project [item_sk,sumss,rk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (23) + Project [item_sk,web_sales,store_sales,rk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (47) - Project [item_sk,web_sales,store_sales,rk] + Window [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt index 204830d57b..f8a95a6db8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt @@ -1,83 +1,80 @@ == Physical Plan == -* CometColumnarToRow (79) -+- CometTakeOrderedAndProject (78) - +- RowToColumnar (77) - +- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * CometColumnarToRow (64) - : +- CometSort (63) - : +- CometExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) - : : +- CometColumnarExchange (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * CometColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) - : +- CometColumnarExchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * CometColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometColumnarExchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * CometColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * CometColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -92,372 +89,360 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(10) CometExchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(10) HashAggregate [codegen id : 3] +(11) CometHashAggregate Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(12) CometExchange +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(12) CometSort -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(13) CometSort +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +(14) CometColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -(14) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(17) CometSort -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] +(18) CometSort +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(18) CometColumnarToRow [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(19) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] +(20) Window +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(20) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] +(22) BroadcastExchange +Input [3]: [item_sk#15, sumws#16, rk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] Join type: Inner -Join condition: (rk#12 >= rk#15) - -(23) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] - -(24) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(25) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] - -(28) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Join condition: (rk#11 >= rk#14) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(28) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] + +(29) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(30) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] + +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(32) CometColumnarToRow [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(33) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#28, d_date#29] +(32) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) -(34) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(35) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +(34) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(36) HashAggregate [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [ss_item_sk#24, d_date#29, sum#31] +(35) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] -(37) CometColumnarExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(36) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] -(38) CometColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] +(37) CometExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(39) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] -Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] -(40) CometColumnarExchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(39) CometExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(41) CometSort -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] +(40) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(43) Window -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] +(42) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] -(44) Project [codegen id : 21] -Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] -Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] +(43) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(46) CometSort -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37], [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(45) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 19] -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(48) Window -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] +(47) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] -(49) Project [codegen id : 20] -Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] -Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] +(48) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] -(50) BroadcastExchange -Input [3]: [item_sk#39, sumss#40, rk#38] +(49) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [item_sk#33] -Right keys [1]: [item_sk#39] +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] Join type: Inner -Join condition: (rk#35 >= rk#38) +Join condition: (rk#32 >= rk#35) -(52) Project [codegen id : 21] -Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] -(53) HashAggregate [codegen id : 21] -Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [partial_sum(sumss#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(54) CometColumnarExchange -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(53) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometColumnarToRow [codegen id : 22] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(56) HashAggregate [codegen id : 22] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [sum(sumss#40)] -Aggregate Attributes [1]: [sum(sumss#40)#45] -Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] +(55) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] -(57) CometColumnarExchange -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(56) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(58) CometSort -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33, d_date#29, cume_sales#46], [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] +(57) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(59) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#10, d_date#6], [item_sk#33, d_date#29], FullOuter +(58) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter -(60) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) +(59) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) -(61) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] +(60) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] -(62) CometExchange -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(63) CometSort -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] +(62) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] -(64) CometColumnarToRow [codegen id : 23] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(65) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(64) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(67) CometSort -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47, d_date#48, web_sales#49, store_sales#50], [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST] +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] -(68) CometColumnarToRow [codegen id : 46] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(69) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(70) Project [codegen id : 47] -Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] -Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] +(69) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] -(71) BroadcastExchange -Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] +(70) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 48] -Left keys [1]: [item_sk#47] -Right keys [1]: [item_sk#53] +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] Join type: Inner -Join condition: (rk#51 >= rk#52) - -(73) Project [codegen id : 48] -Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] - -(74) HashAggregate [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] - -(75) HashAggregate [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [max(web_sales#54), max(store_sales#55)] -Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] - -(76) Filter [codegen id : 48] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(77) RowToColumnar -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] - -(78) CometTakeOrderedAndProject -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#47 ASC NULLS FIRST,d_date#48 ASC NULLS FIRST], output=[item_sk#47,d_date#48,web_sales#49,store_sales#50,web_cumulative#62,store_cumulative#63]), [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63], 100, 0, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] - -(79) CometColumnarToRow [codegen id : 49] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Join condition: (rk#48 >= rk#49) + +(72) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(73) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometProject (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(80) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) +(78) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(82) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(79) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(83) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(84) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt index 898288d55b..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt @@ -1,242 +1,216 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Window expressions are not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometSortMergeJoin + :- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 99 out of 196 eligible operators (50%). Final plan contains 44 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt index 91b14603ab..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt @@ -1,131 +1,116 @@ -WholeStageCodegen (49) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - RowToColumnar - WholeStageCodegen (48) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (24) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (4) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [item_sk,sumws,rk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (6) + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Project [item_sk,sumss,rk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (23) + Project [item_sk,web_sales,store_sales,rk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (47) - Project [item_sk,web_sales,store_sales,rk] + Window [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt index eb446effff..d1daee98c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt @@ -1,57 +1,52 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometTakeOrderedAndProject (52) - +- RowToColumnar (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -69,10 +64,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -80,39 +72,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] +Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] +(8) CometProject Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -(11) ReusedExchange [Reuses operator id: 57] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(10) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) + +(11) CometBroadcastExchange +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [d_date_sk#11, d_year#12, d_moy#13] + +(12) CometBroadcastHashJoin +Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] +Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] +(13) CometProject Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] +Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#14, cc_name#15] @@ -125,188 +124,166 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: [cc_call_center_sk#14, cc_name#15] -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] +Right output [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] +(18) CometProject Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] +Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -(20) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +(20) CometExchange +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +(21) CometHashAggregate +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(22) CometExchange +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(23) CometSort +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(24) CometColumnarToRow [codegen id : 1] +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(26) Filter [codegen id : 2] +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) +(27) Window +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(28) Filter [codegen id : 7] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] +(31) CometHashAggregate +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] +Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] +Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(32) CometExchange +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +(37) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) RowToColumnar -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -(52) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(47) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(53) CometColumnarToRow [codegen id : 23] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(48) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometFilter (55) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) +Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(56) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(57) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index 59b10afc93..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -1,122 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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 + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt index 3340fba52e..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt @@ -1,91 +1,71 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - RowToColumnar - WholeStageCodegen (22) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt index eb446effff..d1daee98c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt @@ -1,57 +1,52 @@ == Physical Plan == -* CometColumnarToRow (53) -+- CometTakeOrderedAndProject (52) - +- RowToColumnar (51) - +- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -69,10 +64,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -80,39 +72,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) -(7) CometColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] +Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] +(8) CometProject Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] +Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -(11) ReusedExchange [Reuses operator id: 57] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(10) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) + +(11) CometBroadcastExchange +Input [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [d_date_sk#11, d_year#12, d_moy#13] + +(12) CometBroadcastHashJoin +Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] +Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] +Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] +(13) CometProject Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] +Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] (14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#14, cc_name#15] @@ -125,188 +124,166 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: [cc_call_center_sk#14, cc_name#15] -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] +Right output [2]: [cc_call_center_sk#14, cc_name#15] +Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] +(18) CometProject Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] +Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -(20) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +(20) CometExchange +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] +(21) CometHashAggregate +Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(22) CometExchange +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(23) CometSort +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(24) CometColumnarToRow [codegen id : 1] +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(26) Filter [codegen id : 2] +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) +(27) Window +Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(28) Filter [codegen id : 7] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] +(31) CometHashAggregate +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] +Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] +Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(32) CometExchange +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +(37) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) RowToColumnar -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -(52) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(47) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(53) CometColumnarToRow [codegen id : 23] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(48) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometFilter (55) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (54) +Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(55) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(56) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(57) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt index 59b10afc93..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt @@ -1,122 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: Window expressions are not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Window expressions are not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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: Window expressions are not supported] + : +- CometColumnarToRow + : +- 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 + : : : +- CometColumnarToRow + : : : +- 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: Window expressions are not supported] + +- CometColumnarToRow + +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 97 eligible operators (44%). Final plan contains 23 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt index 3340fba52e..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt @@ -1,91 +1,71 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - RowToColumnar - WholeStageCodegen (22) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt index c094c4b82b..33c3a5de2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt @@ -1,101 +1,88 @@ == Physical Plan == -* CometColumnarToRow (97) -+- CometTakeOrderedAndProject (96) - +- CometHashAggregate (95) - +- CometColumnarExchange (94) - +- * HashAggregate (93) - +- Union (92) - :- * HashAggregate (77) - : +- * CometColumnarToRow (76) - : +- CometColumnarExchange (75) - : +- * HashAggregate (74) - : +- Union (73) - : :- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometColumnarExchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * CometColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (16) - : : +- * CometColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : :- * HashAggregate (44) - : : +- * CometColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- * HashAggregate (41) - : : +- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * CometColumnarToRow (30) - : : : : +- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (38) - : : +- * CometColumnarToRow (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- Union (58) - : : : :- * CometColumnarToRow (48) - : : : : +- CometProject (47) - : : : : +- CometFilter (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : : +- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildLeft (56) - : : : :- BroadcastExchange (51) - : : : : +- * CometColumnarToRow (50) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (49) - : : : +- * CometColumnarToRow (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : +- ReusedExchange (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometProject (64) - : +- CometFilter (63) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (62) - :- * HashAggregate (84) - : +- * CometColumnarToRow (83) - : +- CometColumnarExchange (82) - : +- * HashAggregate (81) - : +- * HashAggregate (80) - : +- * CometColumnarToRow (79) - : +- ReusedExchange (78) - +- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * HashAggregate (87) - +- * CometColumnarToRow (86) - +- ReusedExchange (85) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -134,465 +121,404 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) CometColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) + +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] -(9) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#22] +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(14) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(15) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#25] +(16) CometProject +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#26] +Arguments: [s_store_sk#24, s_store_id#26] -(17) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#26] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(18) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] +(19) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -(19) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] +(20) CometHashAggregate +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] +Keys [1]: [s_store_id#26] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometExchange +Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] +Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) CometColumnarToRow [codegen id : 4] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(22) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] +(22) CometHashAggregate +Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] +Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] (23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct (24) CometFilter -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) (25) CometProject -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct (27) CometFilter -Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -Condition : isnotnull(cr_catalog_page_sk#54) +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) (28) CometProject -Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] (29) CometUnion -Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] -Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] - -(30) CometColumnarToRow [codegen id : 7] -Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#64] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#49] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) Project [codegen id : 7] -Output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] -Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Condition : isnotnull(cp_catalog_page_sk#65) - -(36) CometProject -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#67], [cp_catalog_page_sk#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#66, 16, true, false, true) AS cp_catalog_page_id#67] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] - -(38) BroadcastExchange -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#48] -Right keys [1]: [cp_catalog_page_sk#65] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] -Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#67] - -(41) HashAggregate [codegen id : 7] -Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] -Keys [1]: [cp_catalog_page_id#67] -Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] -Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] -Results [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] - -(42) CometColumnarExchange -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) CometColumnarToRow [codegen id : 8] -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] - -(44) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] -Keys [1]: [cp_catalog_page_id#67] -Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#76, sum(UnscaledValue(return_amt#52))#77, sum(UnscaledValue(profit#51))#78, sum(UnscaledValue(net_loss#53))#79] -Results [5]: [catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#67) AS id#81, MakeDecimal(sum(UnscaledValue(sales_price#50))#76,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(return_amt#52))#77,17,2) AS returns#83, (MakeDecimal(sum(UnscaledValue(profit#51))#78,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#79,17,2)) AS profit#84] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) + +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] + +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] + +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight + +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] + +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] + +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#88), dynamicpruningexpression(ws_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] -Condition : isnotnull(ws_web_site_sk#85) - -(47) CometProject -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] -Arguments: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95], [ws_web_site_sk#85 AS wsr_web_site_sk#90, ws_sold_date_sk#88 AS date_sk#91, ws_ext_sales_price#86 AS sales_price#92, ws_net_profit#87 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(48) CometColumnarToRow [codegen id : 9] -Input [6]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#100), dynamicpruningexpression(wr_returned_date_sk#100 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(50) CometColumnarToRow [codegen id : 10] -Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] - -(51) BroadcastExchange -Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] -Condition : ((isnotnull(ws_item_sk#101) AND isnotnull(ws_order_number#103)) AND isnotnull(ws_web_site_sk#102)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(54) CometProject -Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] -Arguments: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103], [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(55) CometColumnarToRow -Input [3]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(56) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [wr_item_sk#96, wr_order_number#97] -Right keys [2]: [ws_item_sk#101, ws_order_number#103] -Join type: Inner -Join condition: None +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(57) Project [codegen id : 11] -Output [6]: [ws_web_site_sk#102 AS wsr_web_site_sk#105, wr_returned_date_sk#100 AS date_sk#106, 0.00 AS sales_price#107, 0.00 AS profit#108, wr_return_amt#98 AS return_amt#109, wr_net_loss#99 AS net_loss#110] -Input [8]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100, ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(58) Union +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(59) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#111] +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [date_sk#91] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(61) Project [codegen id : 14] -Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] -Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#111] - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(63) CometFilter -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) - -(64) CometProject -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: [web_site_sk#112, web_site_id#114], [web_site_sk#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#113, 16, true, false, true) AS web_site_id#114] - -(65) CometColumnarToRow [codegen id : 13] -Input [2]: [web_site_sk#112, web_site_id#114] - -(66) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [wsr_web_site_sk#90] -Right keys [1]: [web_site_sk#112] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] -Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#112, web_site_id#114] - -(69) HashAggregate [codegen id : 14] -Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] - -(70) CometColumnarExchange -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(71) CometColumnarToRow [codegen id : 15] -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] - -(72) HashAggregate [codegen id : 15] -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#123, sum(UnscaledValue(return_amt#94))#124, sum(UnscaledValue(profit#93))#125, sum(UnscaledValue(net_loss#95))#126] -Results [5]: [web channel AS channel#127, concat(web_site, web_site_id#114) AS id#128, MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS returns#130, (MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2)) AS profit#131] - -(73) Union - -(74) HashAggregate [codegen id : 16] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(75) CometColumnarExchange -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 17] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(77) HashAggregate [codegen id : 17] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [5]: [channel#38, id#39, cast(sum(sales#40)#144 as decimal(37,2)) AS sales#147, cast(sum(returns#41)#145 as decimal(37,2)) AS returns#148, cast(sum(profit#42)#146 as decimal(38,2)) AS profit#149] - -(78) ReusedExchange [Reuses operator id: 75] -Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(79) CometColumnarToRow [codegen id : 34] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(80) HashAggregate [codegen id : 34] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [4]: [channel#38, sum(sales#40)#144 AS sales#150, sum(returns#41)#145 AS returns#151, sum(profit#42)#146 AS profit#152] - -(81) HashAggregate [codegen id : 34] -Input [4]: [channel#38, sales#150, returns#151, profit#152] -Keys [1]: [channel#38] -Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] -Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -Results [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(82) CometColumnarExchange -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(83) CometColumnarToRow [codegen id : 35] -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(84) HashAggregate [codegen id : 35] -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Keys [1]: [channel#38] -Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] -Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] -Results [5]: [channel#38, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] - -(85) ReusedExchange [Reuses operator id: 75] -Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(86) CometColumnarToRow [codegen id : 52] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(87) HashAggregate [codegen id : 52] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [3]: [sum(sales#40)#144 AS sales#172, sum(returns#41)#145 AS returns#173, sum(profit#42)#146 AS profit#174] - -(88) HashAggregate [codegen id : 52] -Input [3]: [sales#172, returns#173, profit#174] +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] + +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight + +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] + +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(64) CometHashAggregate +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#172), partial_sum(returns#173), partial_sum(profit#174)] -Aggregate Attributes [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Results [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] - -(89) CometColumnarExchange -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] -(90) CometColumnarToRow [codegen id : 53] -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +(77) CometExchange +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(91) HashAggregate [codegen id : 53] -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +(78) CometHashAggregate +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Keys: [] -Functions [3]: [sum(sales#172), sum(returns#173), sum(profit#174)] -Aggregate Attributes [3]: [sum(sales#172)#187, sum(returns#173)#188, sum(profit#174)#189] -Results [5]: [null AS channel#190, null AS id#191, sum(sales#172)#187 AS sum(sales)#192, sum(returns#173)#188 AS sum(returns)#193, sum(profit#174)#189 AS sum(profit)#194] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] -(92) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] +Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] -(93) HashAggregate [codegen id : 54] -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -(94) CometColumnarExchange -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Arguments: hashpartitioning(channel#38, id#39, sales#147, returns#148, profit#149, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(95) CometHashAggregate -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -(96) CometTakeOrderedAndProject -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#147,returns#148,profit#149]), [channel#38, id#39, sales#147, returns#148, profit#149], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#147, returns#148, profit#149] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] -(97) CometColumnarToRow [codegen id : 55] -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (98) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#195] +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(99) CometFilter -Input [2]: [d_date_sk#22, d_date#195] -Condition : (((isnotnull(d_date#195) AND (d_date#195 >= 1998-08-04)) AND (d_date#195 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(86) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(100) CometProject -Input [2]: [d_date_sk#22, d_date#195] +(87) CometProject +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(101) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(102) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#88 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 49 Hosting Expression = wr_returned_date_sk#100 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index 983467d51e..35b86dbff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -1,316 +1,269 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- 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 - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- 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 + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- 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 + : : : : +- 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.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- 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.web_site + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- 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 + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- 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 + : : : : +- 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.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- 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.web_site - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- 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 - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometUnion - : : : :- 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 - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- 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 + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- 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 + : : : +- 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.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- 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.web_site -Comet accelerated 143 out of 263 eligible operators (54%). Final plan contains 51 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt index 3f6eeb8d6d..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt @@ -1,157 +1,99 @@ -WholeStageCodegen (55) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (54) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (17) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (16) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (15) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (14) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (11) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (35) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (34) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (52) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index c094c4b82b..33c3a5de2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -1,101 +1,88 @@ == Physical Plan == -* CometColumnarToRow (97) -+- CometTakeOrderedAndProject (96) - +- CometHashAggregate (95) - +- CometColumnarExchange (94) - +- * HashAggregate (93) - +- Union (92) - :- * HashAggregate (77) - : +- * CometColumnarToRow (76) - : +- CometColumnarExchange (75) - : +- * HashAggregate (74) - : +- Union (73) - : :- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometColumnarExchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * CometColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (16) - : : +- * CometColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : :- * HashAggregate (44) - : : +- * CometColumnarToRow (43) - : : +- CometColumnarExchange (42) - : : +- * HashAggregate (41) - : : +- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * CometColumnarToRow (30) - : : : : +- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (31) - : : +- BroadcastExchange (38) - : : +- * CometColumnarToRow (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- Union (58) - : : : :- * CometColumnarToRow (48) - : : : : +- CometProject (47) - : : : : +- CometFilter (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : : +- * Project (57) - : : : +- * BroadcastHashJoin Inner BuildLeft (56) - : : : :- BroadcastExchange (51) - : : : : +- * CometColumnarToRow (50) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (49) - : : : +- * CometColumnarToRow (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : +- ReusedExchange (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometProject (64) - : +- CometFilter (63) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (62) - :- * HashAggregate (84) - : +- * CometColumnarToRow (83) - : +- CometColumnarExchange (82) - : +- * HashAggregate (81) - : +- * HashAggregate (80) - : +- * CometColumnarToRow (79) - : +- ReusedExchange (78) - +- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * HashAggregate (87) - +- * CometColumnarToRow (86) - +- ReusedExchange (85) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -134,465 +121,404 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) CometColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) + +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] -(9) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#22] +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(14) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(15) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#25] +(16) CometProject +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] -(16) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#26] +Arguments: [s_store_sk#24, s_store_id#26] -(17) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#26] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(18) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] +(19) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -(19) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] +(20) CometHashAggregate +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] +Keys [1]: [s_store_id#26] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(20) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometExchange +Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] +Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) CometColumnarToRow [codegen id : 4] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(22) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] +(22) CometHashAggregate +Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] +Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] (23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct (24) CometFilter -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) (25) CometProject -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct (27) CometFilter -Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -Condition : isnotnull(cr_catalog_page_sk#54) +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) (28) CometProject -Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] -Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] (29) CometUnion -Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] -Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] - -(30) CometColumnarToRow [codegen id : 7] -Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#64] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#49] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) Project [codegen id : 7] -Output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] -Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Condition : isnotnull(cp_catalog_page_sk#65) - -(36) CometProject -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#67], [cp_catalog_page_sk#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#66, 16, true, false, true) AS cp_catalog_page_id#67] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] - -(38) BroadcastExchange -Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#48] -Right keys [1]: [cp_catalog_page_sk#65] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] -Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#67] - -(41) HashAggregate [codegen id : 7] -Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#67] -Keys [1]: [cp_catalog_page_id#67] -Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] -Aggregate Attributes [4]: [sum#68, sum#69, sum#70, sum#71] -Results [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] - -(42) CometColumnarExchange -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) CometColumnarToRow [codegen id : 8] -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] - -(44) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#67, sum#72, sum#73, sum#74, sum#75] -Keys [1]: [cp_catalog_page_id#67] -Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#76, sum(UnscaledValue(return_amt#52))#77, sum(UnscaledValue(profit#51))#78, sum(UnscaledValue(net_loss#53))#79] -Results [5]: [catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#67) AS id#81, MakeDecimal(sum(UnscaledValue(sales_price#50))#76,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(return_amt#52))#77,17,2) AS returns#83, (MakeDecimal(sum(UnscaledValue(profit#51))#78,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#79,17,2)) AS profit#84] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) + +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] + +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] + +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight + +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] + +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] + +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] + +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#88), dynamicpruningexpression(ws_sold_date_sk#88 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] -Condition : isnotnull(ws_web_site_sk#85) - -(47) CometProject -Input [4]: [ws_web_site_sk#85, ws_ext_sales_price#86, ws_net_profit#87, ws_sold_date_sk#88] -Arguments: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95], [ws_web_site_sk#85 AS wsr_web_site_sk#90, ws_sold_date_sk#88 AS date_sk#91, ws_ext_sales_price#86 AS sales_price#92, ws_net_profit#87 AS profit#93, 0.00 AS return_amt#94, 0.00 AS net_loss#95] +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(48) CometColumnarToRow [codegen id : 9] -Input [6]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#100), dynamicpruningexpression(wr_returned_date_sk#100 IN dynamicpruning#89)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(50) CometColumnarToRow [codegen id : 10] -Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] - -(51) BroadcastExchange -Input [5]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] -Condition : ((isnotnull(ws_item_sk#101) AND isnotnull(ws_order_number#103)) AND isnotnull(ws_web_site_sk#102)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(54) CometProject -Input [4]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103, ws_sold_date_sk#104] -Arguments: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103], [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(55) CometColumnarToRow -Input [3]: [ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(56) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [wr_item_sk#96, wr_order_number#97] -Right keys [2]: [ws_item_sk#101, ws_order_number#103] -Join type: Inner -Join condition: None +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(57) Project [codegen id : 11] -Output [6]: [ws_web_site_sk#102 AS wsr_web_site_sk#105, wr_returned_date_sk#100 AS date_sk#106, 0.00 AS sales_price#107, 0.00 AS profit#108, wr_return_amt#98 AS return_amt#109, wr_net_loss#99 AS net_loss#110] -Input [8]: [wr_item_sk#96, wr_order_number#97, wr_return_amt#98, wr_net_loss#99, wr_returned_date_sk#100, ws_item_sk#101, ws_web_site_sk#102, ws_order_number#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(58) Union +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(59) ReusedExchange [Reuses operator id: 102] -Output [1]: [d_date_sk#111] +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [date_sk#91] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(61) Project [codegen id : 14] -Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] -Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#111] - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(63) CometFilter -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) - -(64) CometProject -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: [web_site_sk#112, web_site_id#114], [web_site_sk#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#113, 16, true, false, true) AS web_site_id#114] - -(65) CometColumnarToRow [codegen id : 13] -Input [2]: [web_site_sk#112, web_site_id#114] - -(66) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [wsr_web_site_sk#90] -Right keys [1]: [web_site_sk#112] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] -Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#112, web_site_id#114] - -(69) HashAggregate [codegen id : 14] -Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#92)), partial_sum(UnscaledValue(return_amt#94)), partial_sum(UnscaledValue(profit#93)), partial_sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] - -(70) CometColumnarExchange -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(71) CometColumnarToRow [codegen id : 15] -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] - -(72) HashAggregate [codegen id : 15] -Input [5]: [web_site_id#114, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#92)), sum(UnscaledValue(return_amt#94)), sum(UnscaledValue(profit#93)), sum(UnscaledValue(net_loss#95))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#92))#123, sum(UnscaledValue(return_amt#94))#124, sum(UnscaledValue(profit#93))#125, sum(UnscaledValue(net_loss#95))#126] -Results [5]: [web channel AS channel#127, concat(web_site, web_site_id#114) AS id#128, MakeDecimal(sum(UnscaledValue(sales_price#92))#123,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#94))#124,17,2) AS returns#130, (MakeDecimal(sum(UnscaledValue(profit#93))#125,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#95))#126,17,2)) AS profit#131] - -(73) Union - -(74) HashAggregate [codegen id : 16] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(75) CometColumnarExchange -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 17] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(77) HashAggregate [codegen id : 17] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [5]: [channel#38, id#39, cast(sum(sales#40)#144 as decimal(37,2)) AS sales#147, cast(sum(returns#41)#145 as decimal(37,2)) AS returns#148, cast(sum(profit#42)#146 as decimal(38,2)) AS profit#149] - -(78) ReusedExchange [Reuses operator id: 75] -Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(79) CometColumnarToRow [codegen id : 34] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(80) HashAggregate [codegen id : 34] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [4]: [channel#38, sum(sales#40)#144 AS sales#150, sum(returns#41)#145 AS returns#151, sum(profit#42)#146 AS profit#152] - -(81) HashAggregate [codegen id : 34] -Input [4]: [channel#38, sales#150, returns#151, profit#152] -Keys [1]: [channel#38] -Functions [3]: [partial_sum(sales#150), partial_sum(returns#151), partial_sum(profit#152)] -Aggregate Attributes [6]: [sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -Results [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(82) CometColumnarExchange -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(83) CometColumnarToRow [codegen id : 35] -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(84) HashAggregate [codegen id : 35] -Input [7]: [channel#38, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Keys [1]: [channel#38] -Functions [3]: [sum(sales#150), sum(returns#151), sum(profit#152)] -Aggregate Attributes [3]: [sum(sales#150)#165, sum(returns#151)#166, sum(profit#152)#167] -Results [5]: [channel#38, null AS id#168, sum(sales#150)#165 AS sum(sales)#169, sum(returns#151)#166 AS sum(returns)#170, sum(profit#152)#167 AS sum(profit)#171] - -(85) ReusedExchange [Reuses operator id: 75] -Output [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(86) CometColumnarToRow [codegen id : 52] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(87) HashAggregate [codegen id : 52] -Input [8]: [channel#38, id#39, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#144, sum(returns#41)#145, sum(profit#42)#146] -Results [3]: [sum(sales#40)#144 AS sales#172, sum(returns#41)#145 AS returns#173, sum(profit#42)#146 AS profit#174] - -(88) HashAggregate [codegen id : 52] -Input [3]: [sales#172, returns#173, profit#174] +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] + +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight + +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] + +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(64) CometHashAggregate +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#172), partial_sum(returns#173), partial_sum(profit#174)] -Aggregate Attributes [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Results [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] - -(89) CometColumnarExchange -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] -(90) CometColumnarToRow [codegen id : 53] -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +(77) CometExchange +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(91) HashAggregate [codegen id : 53] -Input [6]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186] +(78) CometHashAggregate +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Keys: [] -Functions [3]: [sum(sales#172), sum(returns#173), sum(profit#174)] -Aggregate Attributes [3]: [sum(sales#172)#187, sum(returns#173)#188, sum(profit#174)#189] -Results [5]: [null AS channel#190, null AS id#191, sum(sales#172)#187 AS sum(sales)#192, sum(returns#173)#188 AS sum(returns)#193, sum(profit#174)#189 AS sum(profit)#194] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] -(92) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] +Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] -(93) HashAggregate [codegen id : 54] -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -(94) CometColumnarExchange -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Arguments: hashpartitioning(channel#38, id#39, sales#147, returns#148, profit#149, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(95) CometHashAggregate -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Keys [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -(96) CometTakeOrderedAndProject -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#147,returns#148,profit#149]), [channel#38, id#39, sales#147, returns#148, profit#149], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#147, returns#148, profit#149] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] -(97) CometColumnarToRow [codegen id : 55] -Input [5]: [channel#38, id#39, sales#147, returns#148, profit#149] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (98) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#195] +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(99) CometFilter -Input [2]: [d_date_sk#22, d_date#195] -Condition : (((isnotnull(d_date#195) AND (d_date#195 >= 1998-08-04)) AND (d_date#195 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +(86) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(100) CometProject -Input [2]: [d_date_sk#22, d_date#195] +(87) CometProject +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(101) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(102) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#88 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 49 Hosting Expression = wr_returned_date_sk#100 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt index 983467d51e..35b86dbff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt @@ -1,316 +1,269 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- 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 - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- 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 + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- 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 + : : : : +- 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.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- 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.web_site + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- 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 + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- 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 + : : : : +- 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.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- 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.web_site - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometUnion - : : : : :- 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 - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometUnion - : : : :- 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 - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- 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 + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- 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 + : : : +- 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.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- 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.web_site -Comet accelerated 143 out of 263 eligible operators (54%). Final plan contains 51 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index 3f6eeb8d6d..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -1,157 +1,99 @@ -WholeStageCodegen (55) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (54) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (17) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (16) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (15) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (14) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (11) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (35) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (34) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (52) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt index c984427acb..73dd1d4955 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt @@ -1,48 +1,44 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- RowToColumnar (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * CometColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (5) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : +- ReusedExchange (17) - +- BroadcastExchange (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * CometColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometFilter (29) - +- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometFilter (38) + +- CometHashAggregate (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) + +- CometBroadcastExchange (29) + +- CometFilter (28) + +- CometHashAggregate (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -60,38 +56,31 @@ Condition : isnotnull(ca_address_sk#1) Input [2]: [ca_address_sk#1, ca_state#2] Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] -(4) CometColumnarToRow [codegen id : 6] -Input [2]: [ca_address_sk#1, ca_state#3] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [2]: [c_customer_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [c_customer_sk#4, c_current_addr_sk#5] Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [c_customer_sk#4, c_current_addr_sk#5] -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#5] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#3] +Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] +Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight -(10) Project [codegen id : 6] -Output [2]: [ca_state#3, c_customer_sk#4] +(8) CometProject Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] +Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -99,224 +88,219 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ca_state#3, c_customer_sk#4] +Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight -(16) Project [codegen id : 6] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +(13) CometProject Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -(17) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#10] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(15) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#11] +Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) -(19) Project [codegen id : 6] -Output [2]: [ca_state#3, ss_item_sk#6] +(16) CometProject +Input [2]: [d_date_sk#10, d_month_seq#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(17) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(18) CometBroadcastHashJoin +Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(19) CometProject Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] (20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] ReadSchema: struct -(24) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) +(23) CometFilter +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) -(25) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] +(24) CometProject +Input [2]: [i_current_price#17, i_category#18] +Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] -(26) CometHashAggregate -Input [2]: [i_current_price#14, i_category#16] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] +(25) CometHashAggregate +Input [2]: [i_current_price#17, i_category#19] +Keys [1]: [i_category#19] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -(27) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(26) CometExchange +Input [3]: [i_category#19, sum#20, count#21] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] +(27) CometHashAggregate +Input [3]: [i_category#19, sum#20, count#21] +Keys [1]: [i_category#19] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] -(29) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) +(28) CometFilter +Input [2]: [avg(i_current_price)#22, i_category#19] +Condition : isnotnull(avg(i_current_price)#22) -(30) CometColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#19, i_category#16] +(29) CometBroadcastExchange +Input [2]: [avg(i_current_price)#22, i_category#19] +Arguments: [avg(i_current_price)#22, i_category#19] -(31) BroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] +(30) CometBroadcastHashJoin +Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Right output [2]: [avg(i_current_price)#22, i_category#19] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight -(32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)] -Right keys [1]: [i_category#16] -Join type: Inner -Join condition: (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)) +(31) CometProject +Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] +Arguments: [i_item_sk#14], [i_item_sk#14] -(33) Project [codegen id : 5] -Output [1]: [i_item_sk#11] -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] +(32) CometBroadcastExchange +Input [1]: [i_item_sk#14] +Arguments: [i_item_sk#14] -(34) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [2]: [ca_state#3, ss_item_sk#6] +Right output [1]: [i_item_sk#14] +Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None +(34) CometProject +Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] +Arguments: [ca_state#3], [ca_state#3] -(36) Project [codegen id : 6] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(37) HashAggregate [codegen id : 6] +(35) CometHashAggregate Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(38) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_state#3, count#21] +(36) CometExchange +Input [2]: [ca_state#3, count#23] +Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) HashAggregate [codegen id : 7] -Input [2]: [ca_state#3, count#21] +(37) CometHashAggregate +Input [2]: [ca_state#3, count#23] Keys [1]: [ca_state#3] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] -(41) Filter [codegen id : 7] -Input [3]: [state#23, cnt#24, ca_state#3] -Condition : (cnt#24 >= 10) +(38) CometFilter +Input [3]: [state#24, cnt#25, ca_state#3] +Condition : (cnt#25 >= 10) -(42) RowToColumnar -Input [3]: [state#23, cnt#24, ca_state#3] +(39) CometTakeOrderedAndProject +Input [3]: [state#24, cnt#25, ca_state#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] -(43) CometTakeOrderedAndProject -Input [3]: [state#23, cnt#24, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#24 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#23,cnt#24]), [state#23, cnt#24], 100, 0, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, cnt#24] - -(44) CometColumnarToRow [codegen id : 8] -Input [2]: [state#23, cnt#24] +(40) CometColumnarToRow [codegen id : 1] +Input [2]: [state#24, cnt#25] ===== Subqueries ===== -Subquery:1 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) +(42) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#11] +Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) -(47) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] +(43) CometProject +Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(48) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(49) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (56) -+- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +* CometColumnarToRow (52) ++- CometHashAggregate (51) + +- CometExchange (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#26, d_year#27, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) +(47) CometFilter +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) -(52) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] +(48) CometProject +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Arguments: [d_month_seq#26], [d_month_seq#26] -(53) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] +(49) CometHashAggregate +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] -(54) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(50) CometExchange +Input [1]: [d_month_seq#26] +Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(55) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] +(51) CometHashAggregate +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] +(52) CometColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#26] + +Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index 3abe831786..f5b69fc6cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -1,72 +1,57 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 36 out of 58 eligible operators (62%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt index 81b3b20040..cbaf71ab0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt @@ -1,81 +1,61 @@ -WholeStageCodegen (8) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] - RowToColumnar - WholeStageCodegen (7) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + CometFilter [state,cnt,ca_state] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] + CometExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #9 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt index c984427acb..73dd1d4955 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt @@ -1,48 +1,44 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- RowToColumnar (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * CometColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (5) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : +- ReusedExchange (17) - +- BroadcastExchange (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * CometColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometFilter (29) - +- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometFilter (38) + +- CometHashAggregate (37) + +- CometExchange (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) + +- CometBroadcastExchange (29) + +- CometFilter (28) + +- CometHashAggregate (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -60,38 +56,31 @@ Condition : isnotnull(ca_address_sk#1) Input [2]: [ca_address_sk#1, ca_state#2] Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] -(4) CometColumnarToRow [codegen id : 6] -Input [2]: [ca_address_sk#1, ca_state#3] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [2]: [c_customer_sk#4, c_current_addr_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [c_customer_sk#4, c_current_addr_sk#5] Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [c_customer_sk#4, c_current_addr_sk#5] -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#5] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#3] +Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] +Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight -(10) Project [codegen id : 6] -Output [2]: [ca_state#3, c_customer_sk#4] +(8) CometProject Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] +Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -99,224 +88,219 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ca_state#3, c_customer_sk#4] +Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight -(16) Project [codegen id : 6] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +(13) CometProject Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] +Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -(17) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#10] +(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(15) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#11] +Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) -(19) Project [codegen id : 6] -Output [2]: [ca_state#3, ss_item_sk#6] +(16) CometProject +Input [2]: [d_date_sk#10, d_month_seq#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(17) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(18) CometBroadcastHashJoin +Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(19) CometProject Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] (20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] +Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] ReadSchema: struct -(24) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) +(23) CometFilter +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) -(25) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] +(24) CometProject +Input [2]: [i_current_price#17, i_category#18] +Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] -(26) CometHashAggregate -Input [2]: [i_current_price#14, i_category#16] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] +(25) CometHashAggregate +Input [2]: [i_current_price#17, i_category#19] +Keys [1]: [i_category#19] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -(27) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(26) CometExchange +Input [3]: [i_category#19, sum#20, count#21] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] +(27) CometHashAggregate +Input [3]: [i_category#19, sum#20, count#21] +Keys [1]: [i_category#19] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] -(29) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) +(28) CometFilter +Input [2]: [avg(i_current_price)#22, i_category#19] +Condition : isnotnull(avg(i_current_price)#22) -(30) CometColumnarToRow [codegen id : 4] -Input [2]: [avg(i_current_price)#19, i_category#16] +(29) CometBroadcastExchange +Input [2]: [avg(i_current_price)#22, i_category#19] +Arguments: [avg(i_current_price)#22, i_category#19] -(31) BroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] +(30) CometBroadcastHashJoin +Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] +Right output [2]: [avg(i_current_price)#22, i_category#19] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight -(32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)] -Right keys [1]: [i_category#16] -Join type: Inner -Join condition: (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)) +(31) CometProject +Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] +Arguments: [i_item_sk#14], [i_item_sk#14] -(33) Project [codegen id : 5] -Output [1]: [i_item_sk#11] -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] +(32) CometBroadcastExchange +Input [1]: [i_item_sk#14] +Arguments: [i_item_sk#14] -(34) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [2]: [ca_state#3, ss_item_sk#6] +Right output [1]: [i_item_sk#14] +Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None +(34) CometProject +Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] +Arguments: [ca_state#3], [ca_state#3] -(36) Project [codegen id : 6] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(37) HashAggregate [codegen id : 6] +(35) CometHashAggregate Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(38) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_state#3, count#21] +(36) CometExchange +Input [2]: [ca_state#3, count#23] +Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) HashAggregate [codegen id : 7] -Input [2]: [ca_state#3, count#21] +(37) CometHashAggregate +Input [2]: [ca_state#3, count#23] Keys [1]: [ca_state#3] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] -(41) Filter [codegen id : 7] -Input [3]: [state#23, cnt#24, ca_state#3] -Condition : (cnt#24 >= 10) +(38) CometFilter +Input [3]: [state#24, cnt#25, ca_state#3] +Condition : (cnt#25 >= 10) -(42) RowToColumnar -Input [3]: [state#23, cnt#24, ca_state#3] +(39) CometTakeOrderedAndProject +Input [3]: [state#24, cnt#25, ca_state#3] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] -(43) CometTakeOrderedAndProject -Input [3]: [state#23, cnt#24, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#24 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#23,cnt#24]), [state#23, cnt#24], 100, 0, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, cnt#24] - -(44) CometColumnarToRow [codegen id : 8] -Input [2]: [state#23, cnt#24] +(40) CometColumnarToRow [codegen id : 1] +Input [2]: [state#24, cnt#25] ===== Subqueries ===== -Subquery:1 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (45) ++- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_month_seq#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) +(42) CometFilter +Input [2]: [d_date_sk#10, d_month_seq#11] +Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) -(47) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] +(43) CometProject +Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(48) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(49) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (56) -+- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +* CometColumnarToRow (52) ++- CometHashAggregate (51) + +- CometExchange (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#26, d_year#27, d_moy#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) +(47) CometFilter +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) -(52) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] +(48) CometProject +Input [3]: [d_month_seq#26, d_year#27, d_moy#28] +Arguments: [d_month_seq#26], [d_month_seq#26] -(53) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] +(49) CometHashAggregate +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] -(54) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(50) CometExchange +Input [1]: [d_month_seq#26] +Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(55) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] +(51) CometHashAggregate +Input [1]: [d_month_seq#26] +Keys [1]: [d_month_seq#26] Functions: [] -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] +(52) CometColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#26] + +Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt index 3abe831786..f5b69fc6cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt @@ -1,72 +1,57 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 36 out of 58 eligible operators (62%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt index 81b3b20040..cbaf71ab0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt @@ -1,81 +1,61 @@ -WholeStageCodegen (8) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] - RowToColumnar - WholeStageCodegen (7) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (6) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + CometFilter [state,cnt,ca_state] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] + CometExchange [ca_state] #1 + CometHashAggregate [ca_state,count] + CometProject [ca_state] + CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk] #7 + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #8 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #9 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt index 753820f88a..077daeca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt @@ -1,191 +1,180 @@ == Physical Plan == -* CometColumnarToRow (187) -+- CometSort (186) - +- CometExchange (185) - +- CometProject (184) - +- CometSortMergeJoin (183) - :- CometSort (113) - : +- CometColumnarExchange (112) - : +- * HashAggregate (111) - : +- * HashAggregate (110) - : +- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Project (102) - : : +- * BroadcastHashJoin Inner BuildRight (101) - : : :- * Project (99) - : : : +- * BroadcastHashJoin Inner BuildRight (98) - : : : :- * Project (93) - : : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : : :- * Project (90) - : : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : : :- * Project (83) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : : : : :- * Project (80) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : : : : :- * Project (74) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : : : :- * Project (68) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : : : : : : : :- * Project (65) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : : : : : :- * Project (58) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : : :- * Project (49) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : : : : : : : : : : :- * Project (43) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (33) - : : : : : : : : : : : : : : : : +- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (41) - : : : : : : : : : : : : : : +- * CometColumnarToRow (40) - : : : : : : : : : : : : : : +- CometProject (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (47) - : : : : : : : : : : : : : +- * CometColumnarToRow (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- BroadcastExchange (53) - : : : : : : : : : : : : +- * CometColumnarToRow (52) - : : : : : : : : : : : : +- CometFilter (51) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - : : : : : : : : : : : +- ReusedExchange (56) - : : : : : : : : : : +- BroadcastExchange (63) - : : : : : : : : : : +- * CometColumnarToRow (62) - : : : : : : : : : : +- CometProject (61) - : : : : : : : : : : +- CometFilter (60) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (59) - : : : : : : : : : +- ReusedExchange (66) - : : : : : : : : +- BroadcastExchange (72) - : : : : : : : : +- * CometColumnarToRow (71) - : : : : : : : : +- CometFilter (70) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (69) - : : : : : : : +- BroadcastExchange (78) - : : : : : : : +- * CometColumnarToRow (77) - : : : : : : : +- CometFilter (76) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (75) - : : : : : : +- ReusedExchange (81) - : : : : : +- BroadcastExchange (88) - : : : : : +- * CometColumnarToRow (87) - : : : : : +- CometProject (86) - : : : : : +- CometFilter (85) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (84) - : : : : +- ReusedExchange (91) - : : : +- BroadcastExchange (97) - : : : +- * CometColumnarToRow (96) - : : : +- CometFilter (95) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (94) - : : +- ReusedExchange (100) - : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) - : +- CometProject (105) - : +- CometFilter (104) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (103) - +- CometSort (182) - +- CometColumnarExchange (181) - +- * HashAggregate (180) - +- * HashAggregate (179) - +- * Project (178) - +- * BroadcastHashJoin Inner BuildRight (177) - :- * Project (175) - : +- * BroadcastHashJoin Inner BuildRight (174) - : :- * Project (172) - : : +- * BroadcastHashJoin Inner BuildRight (171) - : : :- * Project (169) - : : : +- * BroadcastHashJoin Inner BuildRight (168) - : : : :- * Project (166) - : : : : +- * BroadcastHashJoin Inner BuildRight (165) - : : : : :- * Project (163) - : : : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : : : :- * Project (160) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : : : :- * Project (157) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : : : :- * Project (154) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : : : :- * Project (151) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : : : :- * Project (148) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : : : :- * Project (145) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : : : :- * Project (142) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : : : :- * CometColumnarToRow (133) - : : : : : : : : : : : : : : : +- CometProject (132) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (131) - : : : : : : : : : : : : : : : :- CometSort (125) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (124) - : : : : : : : : : : : : : : : : +- * Project (123) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (122) - : : : : : : : : : : : : : : : : :- BroadcastExchange (117) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (116) - : : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (114) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometFilter (119) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (118) - : : : : : : : : : : : : : : : +- CometSort (130) - : : : : : : : : : : : : : : : +- CometProject (129) - : : : : : : : : : : : : : : : +- CometFilter (128) - : : : : : : : : : : : : : : : +- CometHashAggregate (127) - : : : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : : : +- ReusedExchange (146) - : : : : : : : : : +- ReusedExchange (149) - : : : : : : : : +- ReusedExchange (152) - : : : : : : : +- ReusedExchange (155) - : : : : : : +- ReusedExchange (158) - : : : : : +- ReusedExchange (161) - : : : : +- ReusedExchange (164) - : : : +- ReusedExchange (167) - : : +- ReusedExchange (170) - : +- ReusedExchange (173) - +- ReusedExchange (176) +* CometColumnarToRow (176) ++- CometSort (175) + +- CometExchange (174) + +- CometProject (173) + +- CometSortMergeJoin (172) + :- CometSort (103) + : +- CometExchange (102) + : +- CometHashAggregate (101) + : +- CometHashAggregate (100) + : +- CometProject (99) + : +- CometBroadcastHashJoin (98) + : :- CometProject (93) + : : +- CometBroadcastHashJoin (92) + : : :- CometProject (90) + : : : +- CometBroadcastHashJoin (89) + : : : :- CometProject (85) + : : : : +- CometBroadcastHashJoin (84) + : : : : :- CometProject (82) + : : : : : +- CometBroadcastHashJoin (81) + : : : : : :- CometProject (76) + : : : : : : +- CometBroadcastHashJoin (75) + : : : : : : :- CometProject (73) + : : : : : : : +- CometBroadcastHashJoin (72) + : : : : : : : :- CometProject (68) + : : : : : : : : +- CometBroadcastHashJoin (67) + : : : : : : : : :- CometProject (63) + : : : : : : : : : +- CometBroadcastHashJoin (62) + : : : : : : : : : :- CometProject (60) + : : : : : : : : : : +- CometBroadcastHashJoin (59) + : : : : : : : : : : :- CometProject (54) + : : : : : : : : : : : +- CometBroadcastHashJoin (53) + : : : : : : : : : : : :- CometProject (51) + : : : : : : : : : : : : +- CometBroadcastHashJoin (50) + : : : : : : : : : : : : :- CometProject (46) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) + : : : : : : : : : : : : : :- CometProject (41) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (39) + : : : : : : : : : : : : : : +- CometProject (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (44) + : : : : : : : : : : : : : +- CometFilter (43) + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) + : : : : : : : : : : : : +- CometBroadcastExchange (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (52) + : : : : : : : : : : +- CometBroadcastExchange (58) + : : : : : : : : : : +- CometProject (57) + : : : : : : : : : : +- CometFilter (56) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) + : : : : : : : : : +- ReusedExchange (61) + : : : : : : : : +- CometBroadcastExchange (66) + : : : : : : : : +- CometFilter (65) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) + : : : : : : : +- CometBroadcastExchange (71) + : : : : : : : +- CometFilter (70) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) + : : : : : : +- ReusedExchange (74) + : : : : : +- CometBroadcastExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) + : : : : +- ReusedExchange (83) + : : : +- CometBroadcastExchange (88) + : : : +- CometFilter (87) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) + : : +- ReusedExchange (91) + : +- CometBroadcastExchange (97) + : +- CometProject (96) + : +- CometFilter (95) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) + +- CometSort (171) + +- CometExchange (170) + +- CometHashAggregate (169) + +- CometHashAggregate (168) + +- CometProject (167) + +- CometBroadcastHashJoin (166) + :- CometProject (164) + : +- CometBroadcastHashJoin (163) + : :- CometProject (161) + : : +- CometBroadcastHashJoin (160) + : : :- CometProject (158) + : : : +- CometBroadcastHashJoin (157) + : : : :- CometProject (155) + : : : : +- CometBroadcastHashJoin (154) + : : : : :- CometProject (152) + : : : : : +- CometBroadcastHashJoin (151) + : : : : : :- CometProject (149) + : : : : : : +- CometBroadcastHashJoin (148) + : : : : : : :- CometProject (146) + : : : : : : : +- CometBroadcastHashJoin (145) + : : : : : : : :- CometProject (143) + : : : : : : : : +- CometBroadcastHashJoin (142) + : : : : : : : : :- CometProject (140) + : : : : : : : : : +- CometBroadcastHashJoin (139) + : : : : : : : : : :- CometProject (137) + : : : : : : : : : : +- CometBroadcastHashJoin (136) + : : : : : : : : : : :- CometProject (134) + : : : : : : : : : : : +- CometBroadcastHashJoin (133) + : : : : : : : : : : : :- CometProject (131) + : : : : : : : : : : : : +- CometBroadcastHashJoin (130) + : : : : : : : : : : : : :- CometProject (128) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) + : : : : : : : : : : : : : :- CometProject (125) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) + : : : : : : : : : : : : : : :- CometProject (120) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) + : : : : : : : : : : : : : : : :- CometSort (113) + : : : : : : : : : : : : : : : : +- CometExchange (112) + : : : : : : : : : : : : : : : : +- CometProject (111) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) + : : : : : : : : : : : : : : : : : +- CometFilter (105) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometFilter (108) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) + : : : : : : : : : : : : : : : +- CometSort (118) + : : : : : : : : : : : : : : : +- CometProject (117) + : : : : : : : : : : : : : : : +- CometFilter (116) + : : : : : : : : : : : : : : : +- CometHashAggregate (115) + : : : : : : : : : : : : : : : +- ReusedExchange (114) + : : : : : : : : : : : : : : +- CometBroadcastExchange (123) + : : : : : : : : : : : : : : +- CometFilter (122) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) + : : : : : : : : : : : : : +- ReusedExchange (126) + : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : +- ReusedExchange (141) + : : : : : : : +- ReusedExchange (144) + : : : : : : +- ReusedExchange (147) + : : : : : +- ReusedExchange (150) + : : : : +- ReusedExchange (153) + : : : +- ReusedExchange (156) + : : +- ReusedExchange (159) + : +- ReusedExchange (162) + +- ReusedExchange (165) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -200,884 +189,823 @@ ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(7) CometProject +(6) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] +(7) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(11) CometColumnarExchange +(9) CometExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(12) CometSort +(10) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(15) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(16) CometExchange +(14) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(17) CometSort +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) CometExchange +(19) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(22) CometSort +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) CometSortMergeJoin +(21) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) CometProject +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(25) CometHashAggregate +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometExchange +(24) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometHashAggregate +(25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(28) CometFilter +(26) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(29) CometProject +(27) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) CometSort +(28) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) CometSortMergeJoin +(29) CometSortMergeJoin Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [1]: [cs_item_sk#17] Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(32) CometProject +(30) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(33) CometColumnarToRow [codegen id : 18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(34) ReusedExchange [Reuses operator id: 191] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(35) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(36) Project [codegen id : 18] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight + +(35) CometProject Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] ReadSchema: struct -(38) CometFilter +(37) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) -(39) CometProject +(38) CometProject Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] -(40) CometColumnarToRow [codegen id : 4] +(39) CometBroadcastExchange Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] -(41) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] -Join type: Inner -Join condition: None +(40) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(43) Project [codegen id : 18] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] +(41) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) -(46) CometColumnarToRow [codegen id : 5] +(44) CometBroadcastExchange Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(47) BroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] +Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#38] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +(46) CometProject Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(48) CometFilter Input [2]: [d_date_sk#44, d_year#45] Condition : isnotnull(d_date_sk#44) -(52) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#44, d_year#45] - -(53) BroadcastExchange +(49) CometBroadcastExchange Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: [d_date_sk#44, d_year#45] -(54) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#43] -Right keys [1]: [d_date_sk#44] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight -(55) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] +(51) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -(56) ReusedExchange [Reuses operator id: 53] +(52) ReusedExchange [Reuses operator id: 49] Output [2]: [d_date_sk#46, d_year#47] -(57) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#42] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight -(58) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(54) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#48, cd_marital_status#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(60) CometFilter +(56) CometFilter Input [2]: [cd_demo_sk#48, cd_marital_status#49] Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) -(61) CometProject +(57) CometProject Input [2]: [cd_demo_sk#48, cd_marital_status#49] Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] -(62) CometColumnarToRow [codegen id : 8] +(58) CometBroadcastExchange Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#48] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight -(65) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] +(60) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -(66) ReusedExchange [Reuses operator id: 63] +(61) ReusedExchange [Reuses operator id: 58] Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#39] -Right keys [1]: [cd_demo_sk#51] -Join type: Inner -Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) +(62) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] +Right output [2]: [cd_demo_sk#51, cd_marital_status#52] +Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight -(68) Project [codegen id : 18] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(63) CometProject Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(70) CometFilter +(65) CometFilter Input [1]: [p_promo_sk#53] Condition : isnotnull(p_promo_sk#53) -(71) CometColumnarToRow [codegen id : 10] +(66) CometBroadcastExchange Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(72) BroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(67) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [1]: [p_promo_sk#53] +Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight -(73) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#53] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(68) CometProject Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(76) CometFilter +(70) CometFilter Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) -(77) CometColumnarToRow [codegen id : 11] +(71) CometBroadcastExchange Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(78) BroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -(79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#54] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight -(80) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] +(73) CometProject Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -(81) ReusedExchange [Reuses operator id: 78] +(74) ReusedExchange [Reuses operator id: 71] Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#40] -Right keys [1]: [hd_demo_sk#56] -Join type: Inner -Join condition: None +(75) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] +Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight -(83) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] +(76) CometProject Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(85) CometFilter +(78) CometFilter Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Condition : isnotnull(ca_address_sk#58) -(86) CometProject +(79) CometProject Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] -(87) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(88) BroadcastExchange +(80) CometBroadcastExchange Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(89) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#58] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] +Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight -(90) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +(82) CometProject Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(91) ReusedExchange [Reuses operator id: 88] +(83) ReusedExchange [Reuses operator id: 80] Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(92) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#65] -Join type: Inner -Join condition: None +(84) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight -(93) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(85) CometProject Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(95) CometFilter +(87) CometFilter Input [1]: [ib_income_band_sk#70] Condition : isnotnull(ib_income_band_sk#70) -(96) CometColumnarToRow [codegen id : 15] +(88) CometBroadcastExchange Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(97) BroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(89) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [1]: [ib_income_band_sk#70] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#70] -Join type: Inner -Join condition: None - -(99) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(90) CometProject Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(100) ReusedExchange [Reuses operator id: 97] +(91) ReusedExchange [Reuses operator id: 88] Output [1]: [ib_income_band_sk#71] -(101) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(92) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [1]: [ib_income_band_sk#71] +Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight -(102) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(93) CometProject Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(104) CometFilter +(95) CometFilter Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) -(105) CometProject +(96) CometProject Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] -(106) CometColumnarToRow [codegen id : 17] +(97) CometBroadcastExchange Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -(108) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#72] -Join type: Inner -Join condition: None +(98) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [2]: [i_item_sk#72, i_product_name#76] +Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight -(109) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] +(99) CometProject Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -(110) HashAggregate [codegen id : 18] +(100) CometHashAggregate Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] -(111) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] +(101) CometHashAggregate +Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#72 AS item_sk#90, s_store_name#35 AS store_name#91, s_zip#37 AS store_zip#92, ca_street_number#63 AS b_street_number#93, ca_street_name#60 AS b_streen_name#94, ca_city#61 AS b_city#95, ca_zip#64 AS b_zip#96, ca_street_number#66 AS c_street_number#97, ca_street_name#67 AS c_street_name#98, ca_city#68 AS c_city#99, ca_zip#69 AS c_zip#100, d_year#33 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(112) CometColumnarExchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(102) CometExchange +Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(113) CometSort -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] +(103) CometSort +Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] -(114) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(115) CometFilter -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) - -(116) CometColumnarToRow [codegen id : 19] -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(105) CometFilter +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) -(117) BroadcastExchange -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +(106) CometBroadcastExchange +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -(118) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(119) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(108) CometFilter +Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) + +(109) CometProject +Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] + +(110) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [2]: [sr_item_sk#111, sr_ticket_number#112] +Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft + +(111) CometProject +Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] + +(112) CometExchange +Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(113) CometSort +Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] + +(114) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] + +(115) CometHashAggregate +Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] +Keys [1]: [cs_item_sk#114] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] + +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) + +(117) CometProject +Input [3]: [cs_item_sk#114, sale#30, refund#31] +Arguments: [cs_item_sk#114], [cs_item_sk#114] + +(118) CometSort +Input [1]: [cs_item_sk#114] +Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] + +(119) CometSortMergeJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [1]: [cs_item_sk#114] +Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner (120) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] + +(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#122, d_year#123] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(121) CometColumnarToRow -Input [2]: [sr_item_sk#119, sr_ticket_number#120] +(122) CometFilter +Input [2]: [d_date_sk#122, d_year#123] +Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(122) BroadcastHashJoin [codegen id : 20] -Left keys [2]: [ss_item_sk#106, ss_ticket_number#113] -Right keys [2]: [sr_item_sk#119, sr_ticket_number#120] -Join type: Inner -Join condition: None +(123) CometBroadcastExchange +Input [2]: [d_date_sk#122, d_year#123] +Arguments: [d_date_sk#122, d_year#123] -(123) Project [codegen id : 20] -Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +(124) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [2]: [d_date_sk#122, d_year#123] +Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight -(124) CometColumnarExchange -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(125) CometProject +Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -(125) CometSort -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -(126) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(127) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] +Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] +Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight -(127) CometHashAggregate -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +(128) CometProject +Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -(128) CometFilter -Input [3]: [cs_item_sk#122, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] + +(130) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] +Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight + +(131) CometProject +Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#132, d_year#133] + +(133) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Right output [2]: [d_date_sk#132, d_year#133] +Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight + +(134) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#134, d_year#135] + +(136) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] +Right output [2]: [d_date_sk#134, d_year#135] +Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight + +(137) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#136, cd_marital_status#50] + +(139) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [2]: [cd_demo_sk#136, cd_marital_status#50] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight + +(140) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#137, cd_marital_status#52] + +(142) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] +Right output [2]: [cd_demo_sk#137, cd_marital_status#52] +Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight + +(143) CometProject +Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#138] + +(145) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [1]: [p_promo_sk#138] +Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight + +(146) CometProject +Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(129) CometProject -Input [3]: [cs_item_sk#122, sale#30, refund#31] -Arguments: [cs_item_sk#122], [cs_item_sk#122] - -(130) CometSort -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122], [cs_item_sk#122 ASC NULLS FIRST] - -(131) CometSortMergeJoin -Left output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Right output [1]: [cs_item_sk#122] -Arguments: [ss_item_sk#106], [cs_item_sk#122], Inner - -(132) CometProject -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] - -(133) CometColumnarToRow [codegen id : 36] -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] - -(134) ReusedExchange [Reuses operator id: 195] -Output [2]: [d_date_sk#130, d_year#131] - -(135) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None - -(136) Project [codegen id : 36] -Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] - -(137) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#37] - -(138) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None - -(139) Project [codegen id : 36] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37] -Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#37] - -(140) ReusedExchange [Reuses operator id: 47] -Output [6]: [c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] - -(141) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#134] -Join type: Inner -Join condition: None - -(142) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] - -(143) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#140, d_year#141] - -(144) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#139] -Right keys [1]: [d_date_sk#140] -Join type: Inner -Join condition: None - -(145) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139, d_date_sk#140, d_year#141] - -(146) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#142, d_year#143] - -(147) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#138] -Right keys [1]: [d_date_sk#142] -Join type: Inner -Join condition: None - -(148) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141, d_date_sk#142, d_year#143] - -(149) ReusedExchange [Reuses operator id: 63] -Output [2]: [cd_demo_sk#144, cd_marital_status#50] - -(150) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#144] -Join type: Inner -Join condition: None - -(151) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_demo_sk#144, cd_marital_status#50] - -(152) ReusedExchange [Reuses operator id: 63] -Output [2]: [cd_demo_sk#145, cd_marital_status#52] - -(153) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#135] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) - -(154) Project [codegen id : 36] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50, cd_demo_sk#145, cd_marital_status#52] - -(155) ReusedExchange [Reuses operator id: 72] -Output [1]: [p_promo_sk#146] - -(156) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#146] -Join type: Inner -Join condition: None - -(157) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, p_promo_sk#146] - -(158) ReusedExchange [Reuses operator id: 78] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] - -(159) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#147] -Join type: Inner -Join condition: None - -(160) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_demo_sk#147, hd_income_band_sk#148] - -(161) ReusedExchange [Reuses operator id: 78] -Output [2]: [hd_demo_sk#149, hd_income_band_sk#150] - -(162) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#136] -Right keys [1]: [hd_demo_sk#149] -Join type: Inner -Join condition: None - -(163) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_demo_sk#149, hd_income_band_sk#150] - -(164) ReusedExchange [Reuses operator id: 88] -Output [5]: [ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] - -(165) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#151] -Join type: Inner -Join condition: None - -(166) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] - -(167) ReusedExchange [Reuses operator id: 88] -Output [5]: [ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] - -(168) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#137] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None - -(169) Project [codegen id : 36] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] - -(170) ReusedExchange [Reuses operator id: 97] -Output [1]: [ib_income_band_sk#157] - -(171) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#157] -Join type: Inner -Join condition: None - -(172) Project [codegen id : 36] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#157] - -(173) ReusedExchange [Reuses operator id: 97] -Output [1]: [ib_income_band_sk#158] - -(174) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#150] -Right keys [1]: [ib_income_band_sk#158] -Join type: Inner -Join condition: None - -(175) Project [codegen id : 36] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#158] - -(176) ReusedExchange [Reuses operator id: 107] -Output [2]: [i_item_sk#159, i_product_name#76] - -(177) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#159] -Join type: Inner -Join condition: None - -(178) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] - -(179) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#160, sum#161, sum#162] -Results [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] - -(180) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] -Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#159 AS item_sk#166, s_store_name#133 AS store_name#167, s_zip#37 AS store_zip#168, d_year#131 AS syear#169, count(1)#85 AS cnt#170, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#171, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#172, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#173] - -(181) CometColumnarExchange -Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: hashpartitioning(item_sk#166, store_name#167, store_zip#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(182) CometSort -Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173], [item_sk#166 ASC NULLS FIRST, store_name#167 ASC NULLS FIRST, store_zip#168 ASC NULLS FIRST] - -(183) CometSortMergeJoin -Left output [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Right output [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#166, store_name#167, store_zip#168], Inner, (cnt#170 <= cnt#102) - -(184) CometProject -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] - -(185) CometExchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=19] - -(186) CometSort -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] -Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST] - -(187) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +(148) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight + +(149) CometProject +Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] + +(151) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] +Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight + +(152) CometProject +Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] + +(154) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] +Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight + +(155) CometProject +Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#149] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [1]: [ib_income_band_sk#149] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#150] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [1]: [ib_income_band_sk#150] +Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#151, i_product_name#76] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [2]: [i_item_sk#151, i_product_name#76] +Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] + +(169) CometHashAggregate +Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] + +(170) CometExchange +Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] + +(172) CometSortMergeJoin +Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) + +(173) CometProject +Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] + +(174) CometExchange +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] + +(176) CometColumnarToRow [codegen id : 1] +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (191) -+- * CometColumnarToRow (190) - +- CometFilter (189) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (188) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(188) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(189) CometFilter +(178) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(190) CometColumnarToRow [codegen id : 1] +(179) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(191) BroadcastExchange +(180) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (195) -+- * CometColumnarToRow (194) - +- CometFilter (193) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (192) +Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 +BroadcastExchange (184) ++- * CometColumnarToRow (183) + +- CometFilter (182) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) -(192) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#122, d_year#123] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(193) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#122, d_year#123] +Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(194) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#122, d_year#123] -(195) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] +(184) BroadcastExchange +Input [2]: [d_date_sk#122, d_year#123] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index eb776d497d..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -4,280 +4,244 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Project - : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : :- Project - : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : :- Project - : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- 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 + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- 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 - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Project - : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : :- Project - : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : :- Project - : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- 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 138 out of 242 eligible operators (57%). Final plan contains 39 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt index bc50f023fb..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (37) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] @@ -6,260 +6,187 @@ WholeStageCodegen (37) CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometColumnarExchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (18) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + CometExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometColumnarExchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (36) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #19 - WholeStageCodegen (20) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 753820f88a..077daeca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,191 +1,180 @@ == Physical Plan == -* CometColumnarToRow (187) -+- CometSort (186) - +- CometExchange (185) - +- CometProject (184) - +- CometSortMergeJoin (183) - :- CometSort (113) - : +- CometColumnarExchange (112) - : +- * HashAggregate (111) - : +- * HashAggregate (110) - : +- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Project (102) - : : +- * BroadcastHashJoin Inner BuildRight (101) - : : :- * Project (99) - : : : +- * BroadcastHashJoin Inner BuildRight (98) - : : : :- * Project (93) - : : : : +- * BroadcastHashJoin Inner BuildRight (92) - : : : : :- * Project (90) - : : : : : +- * BroadcastHashJoin Inner BuildRight (89) - : : : : : :- * Project (83) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (82) - : : : : : : :- * Project (80) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : : : : :- * Project (74) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : : : :- * Project (68) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : : : : : : : :- * Project (65) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : : : : : :- * Project (58) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : : : : : : : : :- * Project (55) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : : : : : : : : :- * Project (49) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : : : : : : : : : : :- * Project (43) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (42) - : : : : : : : : : : : : : : :- * Project (36) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (33) - : : : : : : : : : : : : : : : : +- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- ReusedExchange (34) - : : : : : : : : : : : : : : +- BroadcastExchange (41) - : : : : : : : : : : : : : : +- * CometColumnarToRow (40) - : : : : : : : : : : : : : : +- CometProject (39) - : : : : : : : : : : : : : : +- CometFilter (38) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (37) - : : : : : : : : : : : : : +- BroadcastExchange (47) - : : : : : : : : : : : : : +- * CometColumnarToRow (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- BroadcastExchange (53) - : : : : : : : : : : : : +- * CometColumnarToRow (52) - : : : : : : : : : : : : +- CometFilter (51) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - : : : : : : : : : : : +- ReusedExchange (56) - : : : : : : : : : : +- BroadcastExchange (63) - : : : : : : : : : : +- * CometColumnarToRow (62) - : : : : : : : : : : +- CometProject (61) - : : : : : : : : : : +- CometFilter (60) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (59) - : : : : : : : : : +- ReusedExchange (66) - : : : : : : : : +- BroadcastExchange (72) - : : : : : : : : +- * CometColumnarToRow (71) - : : : : : : : : +- CometFilter (70) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (69) - : : : : : : : +- BroadcastExchange (78) - : : : : : : : +- * CometColumnarToRow (77) - : : : : : : : +- CometFilter (76) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (75) - : : : : : : +- ReusedExchange (81) - : : : : : +- BroadcastExchange (88) - : : : : : +- * CometColumnarToRow (87) - : : : : : +- CometProject (86) - : : : : : +- CometFilter (85) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (84) - : : : : +- ReusedExchange (91) - : : : +- BroadcastExchange (97) - : : : +- * CometColumnarToRow (96) - : : : +- CometFilter (95) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (94) - : : +- ReusedExchange (100) - : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) - : +- CometProject (105) - : +- CometFilter (104) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (103) - +- CometSort (182) - +- CometColumnarExchange (181) - +- * HashAggregate (180) - +- * HashAggregate (179) - +- * Project (178) - +- * BroadcastHashJoin Inner BuildRight (177) - :- * Project (175) - : +- * BroadcastHashJoin Inner BuildRight (174) - : :- * Project (172) - : : +- * BroadcastHashJoin Inner BuildRight (171) - : : :- * Project (169) - : : : +- * BroadcastHashJoin Inner BuildRight (168) - : : : :- * Project (166) - : : : : +- * BroadcastHashJoin Inner BuildRight (165) - : : : : :- * Project (163) - : : : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : : : :- * Project (160) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : : : :- * Project (157) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : : : :- * Project (154) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : : : :- * Project (151) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : : : :- * Project (148) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : : : :- * Project (145) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : : : :- * Project (142) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : : : :- * CometColumnarToRow (133) - : : : : : : : : : : : : : : : +- CometProject (132) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (131) - : : : : : : : : : : : : : : : :- CometSort (125) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (124) - : : : : : : : : : : : : : : : : +- * Project (123) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (122) - : : : : : : : : : : : : : : : : :- BroadcastExchange (117) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (116) - : : : : : : : : : : : : : : : : : +- CometFilter (115) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (114) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometFilter (119) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (118) - : : : : : : : : : : : : : : : +- CometSort (130) - : : : : : : : : : : : : : : : +- CometProject (129) - : : : : : : : : : : : : : : : +- CometFilter (128) - : : : : : : : : : : : : : : : +- CometHashAggregate (127) - : : : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : : : +- ReusedExchange (146) - : : : : : : : : : +- ReusedExchange (149) - : : : : : : : : +- ReusedExchange (152) - : : : : : : : +- ReusedExchange (155) - : : : : : : +- ReusedExchange (158) - : : : : : +- ReusedExchange (161) - : : : : +- ReusedExchange (164) - : : : +- ReusedExchange (167) - : : +- ReusedExchange (170) - : +- ReusedExchange (173) - +- ReusedExchange (176) +* CometColumnarToRow (176) ++- CometSort (175) + +- CometExchange (174) + +- CometProject (173) + +- CometSortMergeJoin (172) + :- CometSort (103) + : +- CometExchange (102) + : +- CometHashAggregate (101) + : +- CometHashAggregate (100) + : +- CometProject (99) + : +- CometBroadcastHashJoin (98) + : :- CometProject (93) + : : +- CometBroadcastHashJoin (92) + : : :- CometProject (90) + : : : +- CometBroadcastHashJoin (89) + : : : :- CometProject (85) + : : : : +- CometBroadcastHashJoin (84) + : : : : :- CometProject (82) + : : : : : +- CometBroadcastHashJoin (81) + : : : : : :- CometProject (76) + : : : : : : +- CometBroadcastHashJoin (75) + : : : : : : :- CometProject (73) + : : : : : : : +- CometBroadcastHashJoin (72) + : : : : : : : :- CometProject (68) + : : : : : : : : +- CometBroadcastHashJoin (67) + : : : : : : : : :- CometProject (63) + : : : : : : : : : +- CometBroadcastHashJoin (62) + : : : : : : : : : :- CometProject (60) + : : : : : : : : : : +- CometBroadcastHashJoin (59) + : : : : : : : : : : :- CometProject (54) + : : : : : : : : : : : +- CometBroadcastHashJoin (53) + : : : : : : : : : : : :- CometProject (51) + : : : : : : : : : : : : +- CometBroadcastHashJoin (50) + : : : : : : : : : : : : :- CometProject (46) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) + : : : : : : : : : : : : : :- CometProject (41) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) + : : : : : : : : : : : : : : :- CometProject (35) + : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) + : : : : : : : : : : : : : : : :- CometProject (30) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) + : : : : : : : : : : : : : : : : :- CometSort (10) + : : : : : : : : : : : : : : : : : +- CometExchange (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- CometSort (28) + : : : : : : : : : : : : : : : : +- CometProject (27) + : : : : : : : : : : : : : : : : +- CometFilter (26) + : : : : : : : : : : : : : : : : +- CometHashAggregate (25) + : : : : : : : : : : : : : : : : +- CometExchange (24) + : : : : : : : : : : : : : : : : +- CometHashAggregate (23) + : : : : : : : : : : : : : : : : +- CometProject (22) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) + : : : : : : : : : : : : : : : : :- CometSort (15) + : : : : : : : : : : : : : : : : : +- CometExchange (14) + : : : : : : : : : : : : : : : : : +- CometProject (13) + : : : : : : : : : : : : : : : : : +- CometFilter (12) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) + : : : : : : : : : : : : : : : : +- CometSort (20) + : : : : : : : : : : : : : : : : +- CometExchange (19) + : : : : : : : : : : : : : : : : +- CometProject (18) + : : : : : : : : : : : : : : : : +- CometFilter (17) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) + : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) + : : : : : : : : : : : : : : : +- CometFilter (32) + : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) + : : : : : : : : : : : : : : +- CometBroadcastExchange (39) + : : : : : : : : : : : : : : +- CometProject (38) + : : : : : : : : : : : : : : +- CometFilter (37) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) + : : : : : : : : : : : : : +- CometBroadcastExchange (44) + : : : : : : : : : : : : : +- CometFilter (43) + : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) + : : : : : : : : : : : : +- CometBroadcastExchange (49) + : : : : : : : : : : : : +- CometFilter (48) + : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) + : : : : : : : : : : : +- ReusedExchange (52) + : : : : : : : : : : +- CometBroadcastExchange (58) + : : : : : : : : : : +- CometProject (57) + : : : : : : : : : : +- CometFilter (56) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) + : : : : : : : : : +- ReusedExchange (61) + : : : : : : : : +- CometBroadcastExchange (66) + : : : : : : : : +- CometFilter (65) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) + : : : : : : : +- CometBroadcastExchange (71) + : : : : : : : +- CometFilter (70) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) + : : : : : : +- ReusedExchange (74) + : : : : : +- CometBroadcastExchange (80) + : : : : : +- CometProject (79) + : : : : : +- CometFilter (78) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) + : : : : +- ReusedExchange (83) + : : : +- CometBroadcastExchange (88) + : : : +- CometFilter (87) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) + : : +- ReusedExchange (91) + : +- CometBroadcastExchange (97) + : +- CometProject (96) + : +- CometFilter (95) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) + +- CometSort (171) + +- CometExchange (170) + +- CometHashAggregate (169) + +- CometHashAggregate (168) + +- CometProject (167) + +- CometBroadcastHashJoin (166) + :- CometProject (164) + : +- CometBroadcastHashJoin (163) + : :- CometProject (161) + : : +- CometBroadcastHashJoin (160) + : : :- CometProject (158) + : : : +- CometBroadcastHashJoin (157) + : : : :- CometProject (155) + : : : : +- CometBroadcastHashJoin (154) + : : : : :- CometProject (152) + : : : : : +- CometBroadcastHashJoin (151) + : : : : : :- CometProject (149) + : : : : : : +- CometBroadcastHashJoin (148) + : : : : : : :- CometProject (146) + : : : : : : : +- CometBroadcastHashJoin (145) + : : : : : : : :- CometProject (143) + : : : : : : : : +- CometBroadcastHashJoin (142) + : : : : : : : : :- CometProject (140) + : : : : : : : : : +- CometBroadcastHashJoin (139) + : : : : : : : : : :- CometProject (137) + : : : : : : : : : : +- CometBroadcastHashJoin (136) + : : : : : : : : : : :- CometProject (134) + : : : : : : : : : : : +- CometBroadcastHashJoin (133) + : : : : : : : : : : : :- CometProject (131) + : : : : : : : : : : : : +- CometBroadcastHashJoin (130) + : : : : : : : : : : : : :- CometProject (128) + : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) + : : : : : : : : : : : : : :- CometProject (125) + : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) + : : : : : : : : : : : : : : :- CometProject (120) + : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) + : : : : : : : : : : : : : : : :- CometSort (113) + : : : : : : : : : : : : : : : : +- CometExchange (112) + : : : : : : : : : : : : : : : : +- CometProject (111) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) + : : : : : : : : : : : : : : : : : +- CometFilter (105) + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) + : : : : : : : : : : : : : : : : +- CometProject (109) + : : : : : : : : : : : : : : : : +- CometFilter (108) + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) + : : : : : : : : : : : : : : : +- CometSort (118) + : : : : : : : : : : : : : : : +- CometProject (117) + : : : : : : : : : : : : : : : +- CometFilter (116) + : : : : : : : : : : : : : : : +- CometHashAggregate (115) + : : : : : : : : : : : : : : : +- ReusedExchange (114) + : : : : : : : : : : : : : : +- CometBroadcastExchange (123) + : : : : : : : : : : : : : : +- CometFilter (122) + : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) + : : : : : : : : : : : : : +- ReusedExchange (126) + : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : +- ReusedExchange (141) + : : : : : : : +- ReusedExchange (144) + : : : : : : +- ReusedExchange (147) + : : : : : +- ReusedExchange (150) + : : : : +- ReusedExchange (153) + : : : +- ReusedExchange (156) + : : +- ReusedExchange (159) + : +- ReusedExchange (162) + +- ReusedExchange (165) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -200,884 +189,823 @@ ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) -(7) CometProject +(6) CometProject Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] +(7) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +(8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(11) CometColumnarExchange +(9) CometExchange Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(12) CometSort +(10) CometSort Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) -(15) CometProject +(13) CometProject Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -(16) CometExchange +(14) CometExchange Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(17) CometSort +(15) CometSort Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(19) CometFilter +(17) CometFilter Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) -(20) CometProject +(18) CometProject Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(21) CometExchange +(19) CometExchange Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(22) CometSort +(20) CometSort Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] -(23) CometSortMergeJoin +(21) CometSortMergeJoin Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner -(24) CometProject +(22) CometProject Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -(25) CometHashAggregate +(23) CometHashAggregate Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] Keys [1]: [cs_item_sk#17] Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(26) CometExchange +(24) CometExchange Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometHashAggregate +(25) CometHashAggregate Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] Keys [1]: [cs_item_sk#17] Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] -(28) CometFilter +(26) CometFilter Input [3]: [cs_item_sk#17, sale#30, refund#31] Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) -(29) CometProject +(27) CometProject Input [3]: [cs_item_sk#17, sale#30, refund#31] Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) CometSort +(28) CometSort Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) CometSortMergeJoin +(29) CometSortMergeJoin Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [1]: [cs_item_sk#17] Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner -(32) CometProject +(30) CometProject Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -(33) CometColumnarToRow [codegen id : 18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(34) ReusedExchange [Reuses operator id: 191] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(35) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None +(32) CometFilter +Input [2]: [d_date_sk#32, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(36) Project [codegen id : 18] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] + +(34) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight + +(35) CometProject Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] ReadSchema: struct -(38) CometFilter +(37) CometFilter Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) -(39) CometProject +(38) CometProject Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] -(40) CometColumnarToRow [codegen id : 4] +(39) CometBroadcastExchange Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] -(41) BroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#34] -Join type: Inner -Join condition: None +(40) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] +Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight -(43) Project [codegen id : 18] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] +(41) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) -(46) CometColumnarToRow [codegen id : 5] +(44) CometBroadcastExchange Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(47) BroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] +Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#38] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +(46) CometProject Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#44, d_year#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter +(48) CometFilter Input [2]: [d_date_sk#44, d_year#45] Condition : isnotnull(d_date_sk#44) -(52) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#44, d_year#45] - -(53) BroadcastExchange +(49) CometBroadcastExchange Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: [d_date_sk#44, d_year#45] -(54) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#43] -Right keys [1]: [d_date_sk#44] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight -(55) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] +(51) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -(56) ReusedExchange [Reuses operator id: 53] +(52) ReusedExchange [Reuses operator id: 49] Output [2]: [d_date_sk#46, d_year#47] -(57) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#42] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight -(58) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(54) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] +Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#48, cd_marital_status#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(60) CometFilter +(56) CometFilter Input [2]: [cd_demo_sk#48, cd_marital_status#49] Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) -(61) CometProject +(57) CometProject Input [2]: [cd_demo_sk#48, cd_marital_status#49] Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] -(62) CometColumnarToRow [codegen id : 8] +(58) CometBroadcastExchange Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(64) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#48] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight -(65) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] +(60) CometProject Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -(66) ReusedExchange [Reuses operator id: 63] +(61) ReusedExchange [Reuses operator id: 58] Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#39] -Right keys [1]: [cd_demo_sk#51] -Join type: Inner -Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) +(62) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] +Right output [2]: [cd_demo_sk#51, cd_marital_status#52] +Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight -(68) Project [codegen id : 18] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(63) CometProject Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion Output [1]: [p_promo_sk#53] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(70) CometFilter +(65) CometFilter Input [1]: [p_promo_sk#53] Condition : isnotnull(p_promo_sk#53) -(71) CometColumnarToRow [codegen id : 10] +(66) CometBroadcastExchange Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(72) BroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] +(67) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [1]: [p_promo_sk#53] +Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight -(73) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#53] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +(68) CometProject Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] +Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(76) CometFilter +(70) CometFilter Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) -(77) CometColumnarToRow [codegen id : 11] +(71) CometBroadcastExchange Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(78) BroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -(79) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#54] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] +Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight -(80) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] +(73) CometProject Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -(81) ReusedExchange [Reuses operator id: 78] +(74) ReusedExchange [Reuses operator id: 71] Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#40] -Right keys [1]: [hd_demo_sk#56] -Join type: Inner -Join condition: None +(75) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] +Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight -(83) Project [codegen id : 18] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] +(76) CometProject Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] +Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(85) CometFilter +(78) CometFilter Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Condition : isnotnull(ca_address_sk#58) -(86) CometProject +(79) CometProject Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] -(87) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(88) BroadcastExchange +(80) CometBroadcastExchange Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(89) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#58] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] +Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight -(90) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +(82) CometProject Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -(91) ReusedExchange [Reuses operator id: 88] +(83) ReusedExchange [Reuses operator id: 80] Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(92) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#65] -Join type: Inner -Join condition: None +(84) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] +Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight -(93) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(85) CometProject Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band +(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band Output [1]: [ib_income_band_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_income_band_sk)] ReadSchema: struct -(95) CometFilter +(87) CometFilter Input [1]: [ib_income_band_sk#70] Condition : isnotnull(ib_income_band_sk#70) -(96) CometColumnarToRow [codegen id : 15] +(88) CometBroadcastExchange Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(97) BroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(89) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [1]: [ib_income_band_sk#70] +Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#55] -Right keys [1]: [ib_income_band_sk#70] -Join type: Inner -Join condition: None - -(99) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(90) CometProject Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(100) ReusedExchange [Reuses operator id: 97] +(91) ReusedExchange [Reuses operator id: 88] Output [1]: [ib_income_band_sk#71] -(101) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#71] -Join type: Inner -Join condition: None +(92) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [1]: [ib_income_band_sk#71] +Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight -(102) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +(93) CometProject Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] +Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct -(104) CometFilter +(95) CometFilter Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) -(105) CometProject +(96) CometProject Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] -(106) CometColumnarToRow [codegen id : 17] +(97) CometBroadcastExchange Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -(108) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#72] -Join type: Inner -Join condition: None +(98) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] +Right output [2]: [i_item_sk#72, i_product_name#76] +Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight -(109) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] +(99) CometProject Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] +Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -(110) HashAggregate [codegen id : 18] +(100) CometHashAggregate Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] -Results [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] -(111) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#81, sum#82, sum#83, sum#84] +(101) CometHashAggregate +Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] -Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#72 AS item_sk#90, s_store_name#35 AS store_name#91, s_zip#37 AS store_zip#92, ca_street_number#63 AS b_street_number#93, ca_street_name#60 AS b_streen_name#94, ca_city#61 AS b_city#95, ca_zip#64 AS b_zip#96, ca_street_number#66 AS c_street_number#97, ca_street_name#67 AS c_street_name#98, ca_city#68 AS c_city#99, ca_zip#69 AS c_zip#100, d_year#33 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] -(112) CometColumnarExchange -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(102) CometExchange +Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(113) CometSort -Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Arguments: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105], [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST] +(103) CometSort +Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] -(114) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct -(115) CometFilter -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) - -(116) CometColumnarToRow [codegen id : 19] -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +(105) CometFilter +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) -(117) BroadcastExchange -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +(106) CometBroadcastExchange +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -(118) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct -(119) CometFilter -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) +(108) CometFilter +Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) + +(109) CometProject +Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] +Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] + +(110) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [2]: [sr_item_sk#111, sr_ticket_number#112] +Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft + +(111) CometProject +Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] + +(112) CometExchange +Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(113) CometSort +Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] + +(114) ReusedExchange [Reuses operator id: 24] +Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] + +(115) CometHashAggregate +Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] +Keys [1]: [cs_item_sk#114] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] + +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#30, refund#31] +Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) + +(117) CometProject +Input [3]: [cs_item_sk#114, sale#30, refund#31] +Arguments: [cs_item_sk#114], [cs_item_sk#114] + +(118) CometSort +Input [1]: [cs_item_sk#114] +Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] + +(119) CometSortMergeJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [1]: [cs_item_sk#114] +Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner (120) CometProject -Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] -Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] +Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] + +(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#122, d_year#123] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(121) CometColumnarToRow -Input [2]: [sr_item_sk#119, sr_ticket_number#120] +(122) CometFilter +Input [2]: [d_date_sk#122, d_year#123] +Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(122) BroadcastHashJoin [codegen id : 20] -Left keys [2]: [ss_item_sk#106, ss_ticket_number#113] -Right keys [2]: [sr_item_sk#119, sr_ticket_number#120] -Join type: Inner -Join condition: None +(123) CometBroadcastExchange +Input [2]: [d_date_sk#122, d_year#123] +Arguments: [d_date_sk#122, d_year#123] -(123) Project [codegen id : 20] -Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +(124) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] +Right output [2]: [d_date_sk#122, d_year#123] +Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight -(124) CometColumnarExchange -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +(125) CometProject +Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -(125) CometSort -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106 ASC NULLS FIRST] +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -(126) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +(127) CometBroadcastHashJoin +Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] +Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] +Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight -(127) CometHashAggregate -Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -Keys [1]: [cs_item_sk#122] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +(128) CometProject +Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] +Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -(128) CometFilter -Input [3]: [cs_item_sk#122, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] + +(130) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] +Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight + +(131) CometProject +Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#132, d_year#133] + +(133) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] +Right output [2]: [d_date_sk#132, d_year#133] +Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight + +(134) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#134, d_year#135] + +(136) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] +Right output [2]: [d_date_sk#134, d_year#135] +Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight + +(137) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] +Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#136, cd_marital_status#50] + +(139) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [2]: [cd_demo_sk#136, cd_marital_status#50] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight + +(140) CometProject +Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#137, cd_marital_status#52] + +(142) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] +Right output [2]: [cd_demo_sk#137, cd_marital_status#52] +Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight + +(143) CometProject +Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#138] + +(145) CometBroadcastHashJoin +Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [1]: [p_promo_sk#138] +Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight + +(146) CometProject +Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] +Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -(129) CometProject -Input [3]: [cs_item_sk#122, sale#30, refund#31] -Arguments: [cs_item_sk#122], [cs_item_sk#122] - -(130) CometSort -Input [1]: [cs_item_sk#122] -Arguments: [cs_item_sk#122], [cs_item_sk#122 ASC NULLS FIRST] - -(131) CometSortMergeJoin -Left output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Right output [1]: [cs_item_sk#122] -Arguments: [ss_item_sk#106], [cs_item_sk#122], Inner - -(132) CometProject -Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] -Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] - -(133) CometColumnarToRow [codegen id : 36] -Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] - -(134) ReusedExchange [Reuses operator id: 195] -Output [2]: [d_date_sk#130, d_year#131] - -(135) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#117] -Right keys [1]: [d_date_sk#130] -Join type: Inner -Join condition: None - -(136) Project [codegen id : 36] -Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131] -Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#130, d_year#131] - -(137) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#132, s_store_name#133, s_zip#37] - -(138) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#111] -Right keys [1]: [s_store_sk#132] -Join type: Inner -Join condition: None - -(139) Project [codegen id : 36] -Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37] -Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_sk#132, s_store_name#133, s_zip#37] - -(140) ReusedExchange [Reuses operator id: 47] -Output [6]: [c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] - -(141) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#107] -Right keys [1]: [c_customer_sk#134] -Join type: Inner -Join condition: None - -(142) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] -Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_customer_sk#134, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139] - -(143) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#140, d_year#141] - -(144) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#139] -Right keys [1]: [d_date_sk#140] -Join type: Inner -Join condition: None - -(145) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, c_first_sales_date_sk#139, d_date_sk#140, d_year#141] - -(146) ReusedExchange [Reuses operator id: 53] -Output [2]: [d_date_sk#142, d_year#143] - -(147) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#138] -Right keys [1]: [d_date_sk#142] -Join type: Inner -Join condition: None - -(148) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, c_first_shipto_date_sk#138, d_year#141, d_date_sk#142, d_year#143] - -(149) ReusedExchange [Reuses operator id: 63] -Output [2]: [cd_demo_sk#144, cd_marital_status#50] - -(150) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#108] -Right keys [1]: [cd_demo_sk#144] -Join type: Inner -Join condition: None - -(151) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50] -Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_demo_sk#144, cd_marital_status#50] - -(152) ReusedExchange [Reuses operator id: 63] -Output [2]: [cd_demo_sk#145, cd_marital_status#52] - -(153) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#135] -Right keys [1]: [cd_demo_sk#145] -Join type: Inner -Join condition: NOT (cd_marital_status#50 = cd_marital_status#52) - -(154) Project [codegen id : 36] -Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_cdemo_sk#135, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, cd_marital_status#50, cd_demo_sk#145, cd_marital_status#52] - -(155) ReusedExchange [Reuses operator id: 72] -Output [1]: [p_promo_sk#146] - -(156) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#112] -Right keys [1]: [p_promo_sk#146] -Join type: Inner -Join condition: None - -(157) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, p_promo_sk#146] - -(158) ReusedExchange [Reuses operator id: 78] -Output [2]: [hd_demo_sk#147, hd_income_band_sk#148] - -(159) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#109] -Right keys [1]: [hd_demo_sk#147] -Join type: Inner -Join condition: None - -(160) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148] -Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_demo_sk#147, hd_income_band_sk#148] - -(161) ReusedExchange [Reuses operator id: 78] -Output [2]: [hd_demo_sk#149, hd_income_band_sk#150] - -(162) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#136] -Right keys [1]: [hd_demo_sk#149] -Join type: Inner -Join condition: None - -(163) Project [codegen id : 36] -Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150] -Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_hdemo_sk#136, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_demo_sk#149, hd_income_band_sk#150] - -(164) ReusedExchange [Reuses operator id: 88] -Output [5]: [ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] - -(165) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#110] -Right keys [1]: [ca_address_sk#151] -Join type: Inner -Join condition: None - -(166) Project [codegen id : 36] -Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] -Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_address_sk#151, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64] - -(167) ReusedExchange [Reuses operator id: 88] -Output [5]: [ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] - -(168) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#137] -Right keys [1]: [ca_address_sk#154] -Join type: Inner -Join condition: None - -(169) Project [codegen id : 36] -Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, c_current_addr_sk#137, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_address_sk#154, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] - -(170) ReusedExchange [Reuses operator id: 97] -Output [1]: [ib_income_band_sk#157] - -(171) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#148] -Right keys [1]: [ib_income_band_sk#157] -Join type: Inner -Join condition: None - -(172) Project [codegen id : 36] -Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#148, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#157] - -(173) ReusedExchange [Reuses operator id: 97] -Output [1]: [ib_income_band_sk#158] - -(174) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#150] -Right keys [1]: [ib_income_band_sk#158] -Join type: Inner -Join condition: None - -(175) Project [codegen id : 36] -Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, hd_income_band_sk#150, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, ib_income_band_sk#158] - -(176) ReusedExchange [Reuses operator id: 107] -Output [2]: [i_item_sk#159, i_product_name#76] - -(177) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#106] -Right keys [1]: [i_item_sk#159] -Join type: Inner -Join condition: None - -(178) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] -Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, s_store_name#133, s_zip#37, d_year#141, d_year#143, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] - -(179) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#131, d_year#141, d_year#143, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, i_item_sk#159, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count#77, sum#160, sum#161, sum#162] -Results [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] - -(180) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143, count#81, sum#163, sum#164, sum#165] -Keys [15]: [i_product_name#76, i_item_sk#159, s_store_name#133, s_zip#37, ca_street_number#63, ca_street_name#152, ca_city#153, ca_zip#64, ca_street_number#66, ca_street_name#155, ca_city#156, ca_zip#69, d_year#131, d_year#141, d_year#143] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] -Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] -Results [8]: [i_item_sk#159 AS item_sk#166, s_store_name#133 AS store_name#167, s_zip#37 AS store_zip#168, d_year#131 AS syear#169, count(1)#85 AS cnt#170, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#171, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#172, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#173] - -(181) CometColumnarExchange -Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: hashpartitioning(item_sk#166, store_name#167, store_zip#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(182) CometSort -Input [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173], [item_sk#166 ASC NULLS FIRST, store_name#167 ASC NULLS FIRST, store_zip#168 ASC NULLS FIRST] - -(183) CometSortMergeJoin -Left output [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] -Right output [8]: [item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#166, store_name#167, store_zip#168], Inner, (cnt#170 <= cnt#102) - -(184) CometProject -Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#166, store_name#167, store_zip#168, syear#169, cnt#170, s1#171, s2#172, s3#173] -Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] - -(185) CometExchange -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] -Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=19] - -(186) CometSort -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] -Arguments: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170], [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#170 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#171 ASC NULLS FIRST] - -(187) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#171, s2#172, s3#173, syear#169, cnt#170] +(148) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] +Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight + +(149) CometProject +Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] + +(151) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] +Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight + +(152) CometProject +Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] + +(154) CometBroadcastHashJoin +Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] +Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight + +(155) CometProject +Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] +Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#149] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [1]: [ib_income_band_sk#149] +Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#150] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [1]: [ib_income_band_sk#150] +Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#151, i_product_name#76] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] +Right output [2]: [i_item_sk#151, i_product_name#76] +Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] + +(169) CometHashAggregate +Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] + +(170) CometExchange +Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] + +(172) CometSortMergeJoin +Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] +Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) + +(173) CometProject +Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] +Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] + +(174) CometExchange +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(175) CometSort +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] +Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] + +(176) CometColumnarToRow [codegen id : 1] +Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (191) -+- * CometColumnarToRow (190) - +- CometFilter (189) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (188) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(188) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#32, d_year#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(189) CometFilter +(178) CometFilter Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(190) CometColumnarToRow [codegen id : 1] +(179) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] -(191) BroadcastExchange +(180) BroadcastExchange Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 114 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 -BroadcastExchange (195) -+- * CometColumnarToRow (194) - +- CometFilter (193) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (192) +Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 +BroadcastExchange (184) ++- * CometColumnarToRow (183) + +- CometFilter (182) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) -(192) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#130, d_year#131] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#122, d_year#123] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(193) CometFilter -Input [2]: [d_date_sk#130, d_year#131] -Condition : ((isnotnull(d_year#131) AND (d_year#131 = 2000)) AND isnotnull(d_date_sk#130)) +(182) CometFilter +Input [2]: [d_date_sk#122, d_year#123] +Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(194) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#130, d_year#131] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#122, d_year#123] -(195) BroadcastExchange -Input [2]: [d_date_sk#130, d_year#131] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=21] +(184) BroadcastExchange +Input [2]: [d_date_sk#122, d_year#123] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt index eb776d497d..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt @@ -4,280 +4,244 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Project - : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : :- Project - : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : :- Project - : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- 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 + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- 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 - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Project - : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : :- Project - : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : :- Project - : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- 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 138 out of 242 eligible operators (57%). Final plan contains 39 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index bc50f023fb..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (37) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] @@ -6,260 +6,187 @@ WholeStageCodegen (37) CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometColumnarExchange [item_sk,store_name,store_zip] #2 - WholeStageCodegen (18) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + CometExchange [item_sk,store_name,store_zip] #2 + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometExchange [ss_item_sk] #3 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometExchange [cs_item_sk] #6 + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] + CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometColumnarExchange [item_sk,store_name,store_zip] #18 - WholeStageCodegen (36) - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] - HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] - Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] - Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] - Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #19 - WholeStageCodegen (20) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #21 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 - InputAdapter - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [d_date_sk,d_year] #11 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 - InputAdapter - ReusedExchange [p_promo_sk] #13 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [ib_income_band_sk] #16 - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + CometExchange [item_sk,store_name,store_zip] #19 + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] + CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometExchange [ss_item_sk] #20 + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #21 + CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + CometBroadcastExchange [d_date_sk,d_year] #23 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [d_date_sk,d_year] #12 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt index ffe7401317..46f670a0bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt @@ -1,96 +1,78 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- RowToColumnar (90) - +- * Filter (89) - +- Window (88) - +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) - +- CometSort (85) - +- CometColumnarExchange (84) - +- WindowGroupLimit (83) - +- * Sort (82) - +- Union (81) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - :- * HashAggregate (38) - : +- * CometColumnarToRow (37) - : +- CometColumnarExchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- ReusedExchange (39) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- ReusedExchange (46) - :- * HashAggregate (59) - : +- * CometColumnarToRow (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- ReusedExchange (53) - :- * HashAggregate (66) - : +- * CometColumnarToRow (65) - : +- CometColumnarExchange (64) - : +- * HashAggregate (63) - : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) - : +- ReusedExchange (60) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- ReusedExchange (67) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- ReusedExchange (74) +TakeOrderedAndProject (74) ++- * Filter (73) + +- Window (72) + +- WindowGroupLimit (71) + +- * CometColumnarToRow (70) + +- CometSort (69) + +- CometColumnarExchange (68) + +- WindowGroupLimit (67) + +- * CometColumnarToRow (66) + +- CometSort (65) + +- CometUnion (64) + :- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + :- CometHashAggregate (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometHashAggregate (35) + : +- ReusedExchange (34) + :- CometHashAggregate (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometHashAggregate (40) + : +- ReusedExchange (39) + :- CometHashAggregate (48) + : +- CometExchange (47) + : +- CometHashAggregate (46) + : +- CometHashAggregate (45) + : +- ReusedExchange (44) + :- CometHashAggregate (53) + : +- CometExchange (52) + : +- CometHashAggregate (51) + : +- CometHashAggregate (50) + : +- ReusedExchange (49) + :- CometHashAggregate (58) + : +- CometExchange (57) + : +- CometHashAggregate (56) + : +- CometHashAggregate (55) + : +- ReusedExchange (54) + +- CometHashAggregate (63) + +- CometExchange (62) + +- CometHashAggregate (61) + +- CometHashAggregate (60) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -105,452 +87,359 @@ ReadSchema: struct + +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) + +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(4) ReusedExchange [Reuses operator id: 97] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] +(11) CometProject +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] +(12) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#14] +Arguments: [s_store_sk#12, s_store_id#14] -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#14] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] +(16) CometFilter +Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +Condition : isnotnull(i_item_sk#15) -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +(17) CometProject +Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] +Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +(20) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -(21) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +(21) CometHashAggregate +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(22) CometColumnarExchange -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +(23) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] -Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] - -(25) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(26) CometColumnarToRow [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(27) HashAggregate [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(28) HashAggregate [codegen id : 10] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(29) CometColumnarExchange -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(31) HashAggregate [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#49] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] - -(32) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(33) CometColumnarToRow [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(34) HashAggregate [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] - -(35) HashAggregate [codegen id : 16] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [partial_sum(sumsales#59)] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(36) CometColumnarExchange -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(38) HashAggregate [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [sum(sumsales#59)] -Aggregate Attributes [1]: [sum(sumsales#59)#64] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#65, null AS s_store_id#66, sum(sumsales#59)#64 AS sumsales#67] -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(40) CometColumnarToRow [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(41) HashAggregate [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [partial_sum(sumsales#75)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(43) CometColumnarExchange -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(45) HashAggregate [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [sum(sumsales#75)] -Aggregate Attributes [1]: [sum(sumsales#75)#80] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, null AS d_qoy#81, null AS d_moy#82, null AS s_store_id#83, sum(sumsales#75)#80 AS sumsales#84] - -(46) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(47) CometColumnarToRow [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(48) HashAggregate [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] -Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] - -(49) HashAggregate [codegen id : 28] -Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [partial_sum(sumsales#92)] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(50) CometColumnarExchange -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(52) HashAggregate [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [sum(sumsales#92)] -Aggregate Attributes [1]: [sum(sumsales#92)#97] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#92)#97 AS sumsales#102] - -(53) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(54) CometColumnarToRow [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(55) HashAggregate [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] -Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] - -(56) HashAggregate [codegen id : 34] -Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [partial_sum(sumsales#110)] -Aggregate Attributes [2]: [sum#111, isEmpty#112] -Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(57) CometColumnarExchange -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometColumnarToRow [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(59) HashAggregate [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [sum(sumsales#110)] -Aggregate Attributes [1]: [sum(sumsales#110)#115] -Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, null AS d_year#117, null AS d_qoy#118, null AS d_moy#119, null AS s_store_id#120, sum(sumsales#110)#115 AS sumsales#121] - -(60) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(61) CometColumnarToRow [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(62) HashAggregate [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] -Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] - -(63) HashAggregate [codegen id : 40] -Input [3]: [i_category#21, i_class#20, sumsales#129] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [partial_sum(sumsales#129)] -Aggregate Attributes [2]: [sum#130, isEmpty#131] -Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(64) CometColumnarExchange -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometColumnarToRow [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(66) HashAggregate [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [sum(sumsales#129)] -Aggregate Attributes [1]: [sum(sumsales#129)#134] -Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#129)#134 AS sumsales#141] - -(67) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(68) CometColumnarToRow [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(69) HashAggregate [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] -Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] - -(70) HashAggregate [codegen id : 46] -Input [2]: [i_category#21, sumsales#149] -Keys [1]: [i_category#21] -Functions [1]: [partial_sum(sumsales#149)] -Aggregate Attributes [2]: [sum#150, isEmpty#151] -Results [3]: [i_category#21, sum#152, isEmpty#153] - -(71) CometColumnarExchange -Input [3]: [i_category#21, sum#152, isEmpty#153] -Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(72) CometColumnarToRow [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] - -(73) HashAggregate [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] -Keys [1]: [i_category#21] -Functions [1]: [sum(sumsales#149)] -Aggregate Attributes [1]: [sum(sumsales#149)#154] -Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i_product_name#157, null AS d_year#158, null AS d_qoy#159, null AS d_moy#160, null AS s_store_id#161, sum(sumsales#149)#154 AS sumsales#162] - -(74) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(75) CometColumnarToRow [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(76) HashAggregate [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] -Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] - -(77) HashAggregate [codegen id : 52] -Input [1]: [sumsales#170] -Keys: [] -Functions [1]: [partial_sum(sumsales#170)] -Aggregate Attributes [2]: [sum#171, isEmpty#172] -Results [2]: [sum#173, isEmpty#174] +(24) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] + +(25) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] + +(26) CometHashAggregate +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] +Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] +Functions [1]: [partial_sum(sumsales#33)] -(78) CometColumnarExchange -Input [2]: [sum#173, isEmpty#174] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(27) CometExchange +Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(79) CometColumnarToRow [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] +(28) CometHashAggregate +Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] +Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] +Functions [1]: [sum(sumsales#33)] -(80) HashAggregate [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] +(29) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] + +(30) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] + +(31) CometHashAggregate +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] +Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] +Functions [1]: [partial_sum(sumsales#43)] + +(32) CometExchange +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(33) CometHashAggregate +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] +Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] +Functions [1]: [sum(sumsales#43)] + +(34) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] + +(35) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] + +(36) CometHashAggregate +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] +Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] +Functions [1]: [partial_sum(sumsales#53)] + +(37) CometExchange +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(38) CometHashAggregate +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] +Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] +Functions [1]: [sum(sumsales#53)] + +(39) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] + +(40) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] + +(41) CometHashAggregate +Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] +Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] +Functions [1]: [partial_sum(sumsales#63)] + +(42) CometExchange +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(43) CometHashAggregate +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] +Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] +Functions [1]: [sum(sumsales#63)] + +(44) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] + +(45) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] + +(46) CometHashAggregate +Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] +Keys [3]: [i_category#22, i_class#21, i_brand#20] +Functions [1]: [partial_sum(sumsales#73)] + +(47) CometExchange +Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(48) CometHashAggregate +Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] +Keys [3]: [i_category#22, i_class#21, i_brand#20] +Functions [1]: [sum(sumsales#73)] + +(49) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] + +(50) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] + +(51) CometHashAggregate +Input [3]: [i_category#22, i_class#21, sumsales#83] +Keys [2]: [i_category#22, i_class#21] +Functions [1]: [partial_sum(sumsales#83)] + +(52) CometExchange +Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] +Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] +Keys [2]: [i_category#22, i_class#21] +Functions [1]: [sum(sumsales#83)] + +(54) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] + +(55) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] + +(56) CometHashAggregate +Input [2]: [i_category#22, sumsales#93] +Keys [1]: [i_category#22] +Functions [1]: [partial_sum(sumsales#93)] + +(57) CometExchange +Input [3]: [i_category#22, sum#94, isEmpty#95] +Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(58) CometHashAggregate +Input [3]: [i_category#22, sum#94, isEmpty#95] +Keys [1]: [i_category#22] +Functions [1]: [sum(sumsales#93)] + +(59) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] + +(60) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] + +(61) CometHashAggregate +Input [1]: [sumsales#103] Keys: [] -Functions [1]: [sum(sumsales#170)] -Aggregate Attributes [1]: [sum(sumsales#170)#175] -Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#170)#175 AS sumsales#184] +Functions [1]: [partial_sum(sumsales#103)] + +(62) CometExchange +Input [2]: [sum#104, isEmpty#105] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(81) Union +(63) CometHashAggregate +Input [2]: [sum#104, isEmpty#105] +Keys: [] +Functions [1]: [sum(sumsales#103)] -(82) Sort [codegen id : 54] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 +(64) CometUnion +Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] +Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] +Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] +Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] +Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] +Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] +Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] +Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] -(83) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial +(65) CometSort +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] -(84) CometColumnarExchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(66) CometColumnarToRow [codegen id : 1] +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -(85) CometSort -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] +(67) WindowGroupLimit +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial -(86) CometColumnarToRow [codegen id : 55] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +(68) CometColumnarExchange +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(87) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final +(69) CometSort +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] -(88) Window -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] +(70) CometColumnarToRow [codegen id : 2] +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -(89) Filter [codegen id : 56] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Condition : (rk#185 <= 100) +(71) WindowGroupLimit +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final -(90) RowToColumnar -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(72) Window +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] -(91) CometTakeOrderedAndProject -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#28 ASC NULLS FIRST,i_class#29 ASC NULLS FIRST,i_brand#30 ASC NULLS FIRST,i_product_name#31 ASC NULLS FIRST,d_year#32 ASC NULLS FIRST,d_qoy#33 ASC NULLS FIRST,d_moy#34 ASC NULLS FIRST,s_store_id#35 ASC NULLS FIRST,sumsales#36 ASC NULLS FIRST,rk#185 ASC NULLS FIRST], output=[i_category#28,i_class#29,i_brand#30,i_product_name#31,d_year#32,d_qoy#33,d_moy#34,s_store_id#35,sumsales#36,rk#185]), [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185], 100, 0, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#185 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(73) Filter [codegen id : 3] +Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +Condition : (rk#159 <= 100) -(92) CometColumnarToRow [codegen id : 57] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(74) TakeOrderedAndProject +Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (79) ++- * CometColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_month_seq#186 <= 1223)) AND isnotnull(d_date_sk#7)) +(76) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(95) CometProject -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(77) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(96) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(78) CometColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(97) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(79) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index 275e74b65a..da0a1c697a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -1,352 +1,298 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 146 out of 285 eligible operators (51%). Final plan contains 65 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt index a56fc90d3b..6096ee1c25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt @@ -1,151 +1,88 @@ -WholeStageCodegen (57) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - RowToColumnar - WholeStageCodegen (56) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (3) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt index ffe7401317..46f670a0bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt @@ -1,96 +1,78 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- RowToColumnar (90) - +- * Filter (89) - +- Window (88) - +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) - +- CometSort (85) - +- CometColumnarExchange (84) - +- WindowGroupLimit (83) - +- * Sort (82) - +- Union (81) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - :- * HashAggregate (38) - : +- * CometColumnarToRow (37) - : +- CometColumnarExchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- ReusedExchange (39) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- ReusedExchange (46) - :- * HashAggregate (59) - : +- * CometColumnarToRow (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- ReusedExchange (53) - :- * HashAggregate (66) - : +- * CometColumnarToRow (65) - : +- CometColumnarExchange (64) - : +- * HashAggregate (63) - : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) - : +- ReusedExchange (60) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- ReusedExchange (67) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- ReusedExchange (74) +TakeOrderedAndProject (74) ++- * Filter (73) + +- Window (72) + +- WindowGroupLimit (71) + +- * CometColumnarToRow (70) + +- CometSort (69) + +- CometColumnarExchange (68) + +- WindowGroupLimit (67) + +- * CometColumnarToRow (66) + +- CometSort (65) + +- CometUnion (64) + :- CometHashAggregate (23) + : +- CometExchange (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometHashAggregate (25) + : +- ReusedExchange (24) + :- CometHashAggregate (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometHashAggregate (35) + : +- ReusedExchange (34) + :- CometHashAggregate (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometHashAggregate (40) + : +- ReusedExchange (39) + :- CometHashAggregate (48) + : +- CometExchange (47) + : +- CometHashAggregate (46) + : +- CometHashAggregate (45) + : +- ReusedExchange (44) + :- CometHashAggregate (53) + : +- CometExchange (52) + : +- CometHashAggregate (51) + : +- CometHashAggregate (50) + : +- ReusedExchange (49) + :- CometHashAggregate (58) + : +- CometExchange (57) + : +- CometHashAggregate (56) + : +- CometHashAggregate (55) + : +- ReusedExchange (54) + +- CometHashAggregate (63) + +- CometExchange (62) + +- CometHashAggregate (61) + +- CometHashAggregate (60) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -105,452 +87,359 @@ ReadSchema: struct + +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) + +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(4) ReusedExchange [Reuses operator id: 97] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] +(11) CometProject +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] +(12) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#14] +Arguments: [s_store_sk#12, s_store_id#14] -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#14] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] +(16) CometFilter +Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +Condition : isnotnull(i_item_sk#15) -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +(17) CometProject +Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] +Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] +Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] +(20) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -(21) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +(21) CometHashAggregate +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(22) CometColumnarExchange -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(22) CometExchange +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] +(23) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] -Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] - -(25) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(26) CometColumnarToRow [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(27) HashAggregate [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(28) HashAggregate [codegen id : 10] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(29) CometColumnarExchange -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(31) HashAggregate [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#49] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] - -(32) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(33) CometColumnarToRow [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(34) HashAggregate [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] - -(35) HashAggregate [codegen id : 16] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [partial_sum(sumsales#59)] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(36) CometColumnarExchange -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(38) HashAggregate [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [sum(sumsales#59)] -Aggregate Attributes [1]: [sum(sumsales#59)#64] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#65, null AS s_store_id#66, sum(sumsales#59)#64 AS sumsales#67] -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(40) CometColumnarToRow [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(41) HashAggregate [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [partial_sum(sumsales#75)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(43) CometColumnarExchange -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(45) HashAggregate [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [sum(sumsales#75)] -Aggregate Attributes [1]: [sum(sumsales#75)#80] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, null AS d_qoy#81, null AS d_moy#82, null AS s_store_id#83, sum(sumsales#75)#80 AS sumsales#84] - -(46) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(47) CometColumnarToRow [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(48) HashAggregate [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] -Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] - -(49) HashAggregate [codegen id : 28] -Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [partial_sum(sumsales#92)] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(50) CometColumnarExchange -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(52) HashAggregate [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [sum(sumsales#92)] -Aggregate Attributes [1]: [sum(sumsales#92)#97] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#92)#97 AS sumsales#102] - -(53) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(54) CometColumnarToRow [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(55) HashAggregate [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] -Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] - -(56) HashAggregate [codegen id : 34] -Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [partial_sum(sumsales#110)] -Aggregate Attributes [2]: [sum#111, isEmpty#112] -Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(57) CometColumnarExchange -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometColumnarToRow [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(59) HashAggregate [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [sum(sumsales#110)] -Aggregate Attributes [1]: [sum(sumsales#110)#115] -Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, null AS d_year#117, null AS d_qoy#118, null AS d_moy#119, null AS s_store_id#120, sum(sumsales#110)#115 AS sumsales#121] - -(60) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(61) CometColumnarToRow [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(62) HashAggregate [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] -Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] - -(63) HashAggregate [codegen id : 40] -Input [3]: [i_category#21, i_class#20, sumsales#129] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [partial_sum(sumsales#129)] -Aggregate Attributes [2]: [sum#130, isEmpty#131] -Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(64) CometColumnarExchange -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometColumnarToRow [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(66) HashAggregate [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [sum(sumsales#129)] -Aggregate Attributes [1]: [sum(sumsales#129)#134] -Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#129)#134 AS sumsales#141] - -(67) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(68) CometColumnarToRow [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(69) HashAggregate [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] -Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] - -(70) HashAggregate [codegen id : 46] -Input [2]: [i_category#21, sumsales#149] -Keys [1]: [i_category#21] -Functions [1]: [partial_sum(sumsales#149)] -Aggregate Attributes [2]: [sum#150, isEmpty#151] -Results [3]: [i_category#21, sum#152, isEmpty#153] - -(71) CometColumnarExchange -Input [3]: [i_category#21, sum#152, isEmpty#153] -Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(72) CometColumnarToRow [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] - -(73) HashAggregate [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] -Keys [1]: [i_category#21] -Functions [1]: [sum(sumsales#149)] -Aggregate Attributes [1]: [sum(sumsales#149)#154] -Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i_product_name#157, null AS d_year#158, null AS d_qoy#159, null AS d_moy#160, null AS s_store_id#161, sum(sumsales#149)#154 AS sumsales#162] - -(74) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(75) CometColumnarToRow [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(76) HashAggregate [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] -Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] - -(77) HashAggregate [codegen id : 52] -Input [1]: [sumsales#170] -Keys: [] -Functions [1]: [partial_sum(sumsales#170)] -Aggregate Attributes [2]: [sum#171, isEmpty#172] -Results [2]: [sum#173, isEmpty#174] +(24) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] + +(25) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] + +(26) CometHashAggregate +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] +Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] +Functions [1]: [partial_sum(sumsales#33)] -(78) CometColumnarExchange -Input [2]: [sum#173, isEmpty#174] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +(27) CometExchange +Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(79) CometColumnarToRow [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] +(28) CometHashAggregate +Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] +Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] +Functions [1]: [sum(sumsales#33)] -(80) HashAggregate [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] +(29) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] + +(30) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] + +(31) CometHashAggregate +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] +Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] +Functions [1]: [partial_sum(sumsales#43)] + +(32) CometExchange +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(33) CometHashAggregate +Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] +Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] +Functions [1]: [sum(sumsales#43)] + +(34) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] + +(35) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] + +(36) CometHashAggregate +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] +Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] +Functions [1]: [partial_sum(sumsales#53)] + +(37) CometExchange +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(38) CometHashAggregate +Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] +Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] +Functions [1]: [sum(sumsales#53)] + +(39) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] + +(40) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] + +(41) CometHashAggregate +Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] +Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] +Functions [1]: [partial_sum(sumsales#63)] + +(42) CometExchange +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(43) CometHashAggregate +Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] +Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] +Functions [1]: [sum(sumsales#63)] + +(44) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] + +(45) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] + +(46) CometHashAggregate +Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] +Keys [3]: [i_category#22, i_class#21, i_brand#20] +Functions [1]: [partial_sum(sumsales#73)] + +(47) CometExchange +Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] +Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(48) CometHashAggregate +Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] +Keys [3]: [i_category#22, i_class#21, i_brand#20] +Functions [1]: [sum(sumsales#73)] + +(49) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] + +(50) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] + +(51) CometHashAggregate +Input [3]: [i_category#22, i_class#21, sumsales#83] +Keys [2]: [i_category#22, i_class#21] +Functions [1]: [partial_sum(sumsales#83)] + +(52) CometExchange +Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] +Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(53) CometHashAggregate +Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] +Keys [2]: [i_category#22, i_class#21] +Functions [1]: [sum(sumsales#83)] + +(54) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] + +(55) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] + +(56) CometHashAggregate +Input [2]: [i_category#22, sumsales#93] +Keys [1]: [i_category#22] +Functions [1]: [partial_sum(sumsales#93)] + +(57) CometExchange +Input [3]: [i_category#22, sum#94, isEmpty#95] +Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(58) CometHashAggregate +Input [3]: [i_category#22, sum#94, isEmpty#95] +Keys [1]: [i_category#22] +Functions [1]: [sum(sumsales#93)] + +(59) ReusedExchange [Reuses operator id: 22] +Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] + +(60) CometHashAggregate +Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] +Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] +Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] + +(61) CometHashAggregate +Input [1]: [sumsales#103] Keys: [] -Functions [1]: [sum(sumsales#170)] -Aggregate Attributes [1]: [sum(sumsales#170)#175] -Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#170)#175 AS sumsales#184] +Functions [1]: [partial_sum(sumsales#103)] + +(62) CometExchange +Input [2]: [sum#104, isEmpty#105] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(81) Union +(63) CometHashAggregate +Input [2]: [sum#104, isEmpty#105] +Keys: [] +Functions [1]: [sum(sumsales#103)] -(82) Sort [codegen id : 54] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 +(64) CometUnion +Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] +Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] +Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] +Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] +Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] +Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] +Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] +Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] -(83) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial +(65) CometSort +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] -(84) CometColumnarExchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(66) CometColumnarToRow [codegen id : 1] +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -(85) CometSort -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] +(67) WindowGroupLimit +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial -(86) CometColumnarToRow [codegen id : 55] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] +(68) CometColumnarExchange +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(87) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final +(69) CometSort +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] -(88) Window -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] +(70) CometColumnarToRow [codegen id : 2] +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -(89) Filter [codegen id : 56] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Condition : (rk#185 <= 100) +(71) WindowGroupLimit +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final -(90) RowToColumnar -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(72) Window +Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] +Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] -(91) CometTakeOrderedAndProject -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#28 ASC NULLS FIRST,i_class#29 ASC NULLS FIRST,i_brand#30 ASC NULLS FIRST,i_product_name#31 ASC NULLS FIRST,d_year#32 ASC NULLS FIRST,d_qoy#33 ASC NULLS FIRST,d_moy#34 ASC NULLS FIRST,s_store_id#35 ASC NULLS FIRST,sumsales#36 ASC NULLS FIRST,rk#185 ASC NULLS FIRST], output=[i_category#28,i_class#29,i_brand#30,i_product_name#31,d_year#32,d_qoy#33,d_moy#34,s_store_id#35,sumsales#36,rk#185]), [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185], 100, 0, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#185 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(73) Filter [codegen id : 3] +Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +Condition : (rk#159 <= 100) -(92) CometColumnarToRow [codegen id : 57] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] +(74) TakeOrderedAndProject +Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] +Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (79) ++- * CometColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] +(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_month_seq#186 <= 1223)) AND isnotnull(d_date_sk#7)) +(76) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(95) CometProject -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(77) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(96) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(78) CometColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(97) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(79) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt index 275e74b65a..da0a1c697a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt @@ -1,352 +1,298 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 146 out of 285 eligible operators (51%). Final plan contains 65 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt index a56fc90d3b..6096ee1c25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt @@ -1,151 +1,88 @@ -WholeStageCodegen (57) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - RowToColumnar - WholeStageCodegen (56) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (3) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt index 82f75a1a91..af71da5f87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt @@ -1,71 +1,70 @@ == Physical Plan == -* CometColumnarToRow (67) -+- CometTakeOrderedAndProject (66) - +- RowToColumnar (65) - +- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- WindowGroupLimit (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * CometColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- ReusedExchange (20) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) +TakeOrderedAndProject (66) ++- * Project (65) + +- Window (64) + +- * CometColumnarToRow (63) + +- CometSort (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometColumnarExchange (59) + +- * HashAggregate (58) + +- Union (57) + :- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometColumnarExchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * BroadcastHashJoin LeftSemi BuildRight (34) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- Window (30) + : +- WindowGroupLimit (29) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) + : +- CometExchange (25) + : +- CometHashAggregate (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- ReusedExchange (21) + :- * HashAggregate (49) + : +- * CometColumnarToRow (48) + : +- CometColumnarExchange (47) + : +- * HashAggregate (46) + : +- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- ReusedExchange (43) + +- * HashAggregate (56) + +- * CometColumnarToRow (55) + +- CometColumnarExchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * CometColumnarToRow (51) + +- ReusedExchange (50) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -80,333 +79,330 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13, s_state#15], [s_store_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) AS s_state#15] +(17) CometProject +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#15] +(18) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#16] +Arguments: [s_store_sk#14, s_state#16] -(17) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(19) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#16] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(20) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#15] +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#17] -(20) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#16] +(22) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] +Right output [1]: [d_date_sk#17] +Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(23) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] +Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] + +(24) CometHashAggregate +Input [2]: [ss_net_profit#11, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] + +(25) CometExchange +Input [2]: [s_state#16, sum#18] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(26) CometHashAggregate +Input [2]: [s_state#16, sum#18] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] + +(27) CometSort +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#15] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15, d_date_sk#16] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#15] -Keys [1]: [s_state#15] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#17] -Results [2]: [s_state#15, sum#18] - -(24) CometColumnarExchange -Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#15, sum#18] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#15, sum#18] -Keys [1]: [s_state#15] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#19] -Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#19,17,2) AS _w0#20, s_state#15] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final - -(29) Window -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -Condition : (ranking#21 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] - -(32) BroadcastExchange -Input [1]: [s_state#15] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#15] +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] + +(29) WindowGroupLimit +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final + +(30) Window +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] + +(31) Filter [codegen id : 2] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +Condition : (ranking#20 <= 5) + +(32) Project [codegen id : 2] +Output [1]: [s_state#16] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] + +(33) BroadcastExchange +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(34) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#16] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#22] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(35) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(36) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#22] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#22] +(38) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_county#8, s_state#21] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#22] -Keys [2]: [s_state#22, s_county#7] +(39) HashAggregate [codegen id : 4] +Input [3]: [ss_net_profit#2, s_county#8, s_state#21] +Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [s_state#22, s_county#7, sum#24] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] -(39) CometColumnarExchange -Input [3]: [s_state#22, s_county#7, sum#24] -Arguments: hashpartitioning(s_state#22, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(40) CometColumnarExchange +Input [3]: [s_state#21, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 9] -Input [3]: [s_state#22, s_county#7, sum#24] +(41) CometColumnarToRow [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] -(41) HashAggregate [codegen id : 9] -Input [3]: [s_state#22, s_county#7, sum#24] -Keys [2]: [s_state#22, s_county#7] +(42) HashAggregate [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] +Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#22 AS s_state#27, s_county#7 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#22, s_county#32, sum#33] - -(43) CometColumnarToRow [codegen id : 18] -Input [3]: [s_state#22, s_county#32, sum#33] - -(44) HashAggregate [codegen id : 18] -Input [3]: [s_state#22, s_county#32, sum#33] -Keys [2]: [s_state#22, s_county#32] -Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#25] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#25,17,2) AS total_sum#35, s_state#22] - -(45) HashAggregate [codegen id : 18] -Input [2]: [total_sum#35, s_state#22] -Keys [1]: [s_state#22] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [3]: [s_state#22, sum#38, isEmpty#39] - -(46) CometColumnarExchange -Input [3]: [s_state#22, sum#38, isEmpty#39] -Arguments: hashpartitioning(s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(47) CometColumnarToRow [codegen id : 19] -Input [3]: [s_state#22, sum#38, isEmpty#39] - -(48) HashAggregate [codegen id : 19] -Input [3]: [s_state#22, sum#38, isEmpty#39] -Keys [1]: [s_state#22] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#22, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#22, s_county#46, sum#47] - -(50) CometColumnarToRow [codegen id : 28] -Input [3]: [s_state#22, s_county#46, sum#47] - -(51) HashAggregate [codegen id : 28] -Input [3]: [s_state#22, s_county#46, sum#47] -Keys [2]: [s_state#22, s_county#46] -Functions [1]: [sum(UnscaledValue(ss_net_profit#48))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#48))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#48))#25,17,2) AS total_sum#49] - -(52) HashAggregate [codegen id : 28] -Input [1]: [total_sum#49] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] + +(43) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_state#21, s_county#31, sum#32] + +(44) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] + +(45) HashAggregate [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] +Keys [2]: [s_state#21, s_county#31] +Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] + +(46) HashAggregate [codegen id : 10] +Input [2]: [total_sum#34, s_state#21] +Keys [1]: [s_state#21] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [3]: [s_state#21, sum#37, isEmpty#38] + +(47) CometColumnarExchange +Input [3]: [s_state#21, sum#37, isEmpty#38] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(48) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] + +(49) HashAggregate [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] +Keys [1]: [s_state#21] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#39] +Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] + +(50) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_state#21, s_county#45, sum#46] + +(51) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] + +(52) HashAggregate [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] +Keys [2]: [s_state#21, s_county#45] +Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] + +(53) HashAggregate [codegen id : 16] +Input [1]: [total_sum#48] Keys: [] -Functions [1]: [partial_sum(total_sum#49)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [2]: [sum#52, isEmpty#53] +Functions [1]: [partial_sum(total_sum#48)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [2]: [sum#51, isEmpty#52] -(53) CometColumnarExchange -Input [2]: [sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(54) CometColumnarExchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(54) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#52, isEmpty#53] +(55) CometColumnarToRow [codegen id : 17] +Input [2]: [sum#51, isEmpty#52] -(55) HashAggregate [codegen id : 29] -Input [2]: [sum#52, isEmpty#53] +(56) HashAggregate [codegen id : 17] +Input [2]: [sum#51, isEmpty#52] Keys: [] -Functions [1]: [sum(total_sum#49)] -Aggregate Attributes [1]: [sum(total_sum#49)#54] -Results [6]: [sum(total_sum#49)#54 AS total_sum#55, null AS s_state#56, null AS s_county#57, 1 AS g_state#58, 1 AS g_county#59, 2 AS lochierarchy#60] +Functions [1]: [sum(total_sum#48)] +Aggregate Attributes [1]: [sum(total_sum#48)#53] +Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] -(56) Union +(57) Union -(57) HashAggregate [codegen id : 30] -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +(58) HashAggregate [codegen id : 18] +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(58) CometColumnarExchange -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Arguments: hashpartitioning(total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(59) CometColumnarExchange +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometHashAggregate -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +(60) CometHashAggregate +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] -(60) CometExchange -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: hashpartitioning(lochierarchy#31, _w0#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometSort -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61], [lochierarchy#31 ASC NULLS FIRST, _w0#61 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 31] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +(61) CometExchange +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(63) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#61, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#62], [lochierarchy#31, _w0#61], [total_sum#26 DESC NULLS LAST] +(62) CometSort +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(64) Project [codegen id : 32] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61, rank_within_parent#62] +(63) CometColumnarToRow [codegen id : 19] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -(65) RowToColumnar -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(64) Window +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] -(66) CometTakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#62 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#62]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#62 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(65) Project [codegen id : 20] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] -(67) CometColumnarToRow [codegen id : 33] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(66) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#63] +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) +(68) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject -Input [2]: [d_date_sk#5, d_month_seq#63] +(69) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(71) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(72) BroadcastExchange +(71) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index dafedcf28b..1574f09c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -1,190 +1,176 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- 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 - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- 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 67 out of 156 eligible operators (42%). Final plan contains 32 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt index e8d5239ead..2832c5a348 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt @@ -1,111 +1,101 @@ -WholeStageCodegen (33) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (32) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt index 82f75a1a91..af71da5f87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt @@ -1,71 +1,70 @@ == Physical Plan == -* CometColumnarToRow (67) -+- CometTakeOrderedAndProject (66) - +- RowToColumnar (65) - +- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- WindowGroupLimit (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * CometColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- ReusedExchange (20) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) +TakeOrderedAndProject (66) ++- * Project (65) + +- Window (64) + +- * CometColumnarToRow (63) + +- CometSort (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometColumnarExchange (59) + +- * HashAggregate (58) + +- Union (57) + :- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometColumnarExchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (36) + : +- * Project (35) + : +- * BroadcastHashJoin LeftSemi BuildRight (34) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- Window (30) + : +- WindowGroupLimit (29) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) + : +- CometExchange (25) + : +- CometHashAggregate (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- ReusedExchange (21) + :- * HashAggregate (49) + : +- * CometColumnarToRow (48) + : +- CometColumnarExchange (47) + : +- * HashAggregate (46) + : +- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- ReusedExchange (43) + +- * HashAggregate (56) + +- * CometColumnarToRow (55) + +- CometColumnarExchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- * CometColumnarToRow (51) + +- ReusedExchange (50) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -80,333 +79,330 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13, s_state#15], [s_store_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) AS s_state#15] +(17) CometProject +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#15] +(18) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#16] +Arguments: [s_store_sk#14, s_state#16] -(17) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(19) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#16] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(20) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#15] +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#17] -(20) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#16] +(22) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] +Right output [1]: [d_date_sk#17] +Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(23) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] +Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] + +(24) CometHashAggregate +Input [2]: [ss_net_profit#11, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] + +(25) CometExchange +Input [2]: [s_state#16, sum#18] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] + +(26) CometHashAggregate +Input [2]: [s_state#16, sum#18] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] + +(27) CometSort +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#15] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#15, d_date_sk#16] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#15] -Keys [1]: [s_state#15] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#17] -Results [2]: [s_state#15, sum#18] - -(24) CometColumnarExchange -Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#15, sum#18] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#15, sum#18] -Keys [1]: [s_state#15] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#19] -Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#19,17,2) AS _w0#20, s_state#15] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final - -(29) Window -Input [3]: [s_state#15, _w0#20, s_state#15] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -Condition : (ranking#21 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#15] -Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] - -(32) BroadcastExchange -Input [1]: [s_state#15] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#15] +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] + +(29) WindowGroupLimit +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final + +(30) Window +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] + +(31) Filter [codegen id : 2] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +Condition : (ranking#20 <= 5) + +(32) Project [codegen id : 2] +Output [1]: [s_state#16] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] + +(33) BroadcastExchange +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(34) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#16] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#22] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(35) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(36) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(36) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#22] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#22] +(38) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_county#8, s_state#21] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#22] -Keys [2]: [s_state#22, s_county#7] +(39) HashAggregate [codegen id : 4] +Input [3]: [ss_net_profit#2, s_county#8, s_state#21] +Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [s_state#22, s_county#7, sum#24] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] -(39) CometColumnarExchange -Input [3]: [s_state#22, s_county#7, sum#24] -Arguments: hashpartitioning(s_state#22, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(40) CometColumnarExchange +Input [3]: [s_state#21, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 9] -Input [3]: [s_state#22, s_county#7, sum#24] +(41) CometColumnarToRow [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] -(41) HashAggregate [codegen id : 9] -Input [3]: [s_state#22, s_county#7, sum#24] -Keys [2]: [s_state#22, s_county#7] +(42) HashAggregate [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] +Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#22 AS s_state#27, s_county#7 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#22, s_county#32, sum#33] - -(43) CometColumnarToRow [codegen id : 18] -Input [3]: [s_state#22, s_county#32, sum#33] - -(44) HashAggregate [codegen id : 18] -Input [3]: [s_state#22, s_county#32, sum#33] -Keys [2]: [s_state#22, s_county#32] -Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#25] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#25,17,2) AS total_sum#35, s_state#22] - -(45) HashAggregate [codegen id : 18] -Input [2]: [total_sum#35, s_state#22] -Keys [1]: [s_state#22] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [3]: [s_state#22, sum#38, isEmpty#39] - -(46) CometColumnarExchange -Input [3]: [s_state#22, sum#38, isEmpty#39] -Arguments: hashpartitioning(s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(47) CometColumnarToRow [codegen id : 19] -Input [3]: [s_state#22, sum#38, isEmpty#39] - -(48) HashAggregate [codegen id : 19] -Input [3]: [s_state#22, sum#38, isEmpty#39] -Keys [1]: [s_state#22] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#22, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#22, s_county#46, sum#47] - -(50) CometColumnarToRow [codegen id : 28] -Input [3]: [s_state#22, s_county#46, sum#47] - -(51) HashAggregate [codegen id : 28] -Input [3]: [s_state#22, s_county#46, sum#47] -Keys [2]: [s_state#22, s_county#46] -Functions [1]: [sum(UnscaledValue(ss_net_profit#48))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#48))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#48))#25,17,2) AS total_sum#49] - -(52) HashAggregate [codegen id : 28] -Input [1]: [total_sum#49] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] + +(43) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_state#21, s_county#31, sum#32] + +(44) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] + +(45) HashAggregate [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] +Keys [2]: [s_state#21, s_county#31] +Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] + +(46) HashAggregate [codegen id : 10] +Input [2]: [total_sum#34, s_state#21] +Keys [1]: [s_state#21] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [3]: [s_state#21, sum#37, isEmpty#38] + +(47) CometColumnarExchange +Input [3]: [s_state#21, sum#37, isEmpty#38] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(48) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] + +(49) HashAggregate [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] +Keys [1]: [s_state#21] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#39] +Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] + +(50) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_state#21, s_county#45, sum#46] + +(51) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] + +(52) HashAggregate [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] +Keys [2]: [s_state#21, s_county#45] +Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] + +(53) HashAggregate [codegen id : 16] +Input [1]: [total_sum#48] Keys: [] -Functions [1]: [partial_sum(total_sum#49)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [2]: [sum#52, isEmpty#53] +Functions [1]: [partial_sum(total_sum#48)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [2]: [sum#51, isEmpty#52] -(53) CometColumnarExchange -Input [2]: [sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(54) CometColumnarExchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(54) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#52, isEmpty#53] +(55) CometColumnarToRow [codegen id : 17] +Input [2]: [sum#51, isEmpty#52] -(55) HashAggregate [codegen id : 29] -Input [2]: [sum#52, isEmpty#53] +(56) HashAggregate [codegen id : 17] +Input [2]: [sum#51, isEmpty#52] Keys: [] -Functions [1]: [sum(total_sum#49)] -Aggregate Attributes [1]: [sum(total_sum#49)#54] -Results [6]: [sum(total_sum#49)#54 AS total_sum#55, null AS s_state#56, null AS s_county#57, 1 AS g_state#58, 1 AS g_county#59, 2 AS lochierarchy#60] +Functions [1]: [sum(total_sum#48)] +Aggregate Attributes [1]: [sum(total_sum#48)#53] +Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] -(56) Union +(57) Union -(57) HashAggregate [codegen id : 30] -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +(58) HashAggregate [codegen id : 18] +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(58) CometColumnarExchange -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Arguments: hashpartitioning(total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +(59) CometColumnarExchange +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometHashAggregate -Input [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] -Keys [6]: [total_sum#26, s_state#27, s_county#28, g_state#29, g_county#30, lochierarchy#31] +(60) CometHashAggregate +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] -(60) CometExchange -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: hashpartitioning(lochierarchy#31, _w0#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometSort -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61], [lochierarchy#31 ASC NULLS FIRST, _w0#61 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 31] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] +(61) CometExchange +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(63) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#61, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#62], [lochierarchy#31, _w0#61], [total_sum#26 DESC NULLS LAST] +(62) CometSort +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(64) Project [codegen id : 32] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#61, rank_within_parent#62] +(63) CometColumnarToRow [codegen id : 19] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -(65) RowToColumnar -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(64) Window +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] -(66) CometTakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#62 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#62]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#62 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(65) Project [codegen id : 20] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] -(67) CometColumnarToRow [codegen id : 33] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#62] +(66) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#63] +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) +(68) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject -Input [2]: [d_date_sk#5, d_month_seq#63] +(69) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(71) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(72) BroadcastExchange +(71) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt index dafedcf28b..1574f09c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt @@ -1,190 +1,176 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- 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 - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- 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 67 out of 156 eligible operators (42%). Final plan contains 32 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index e8d5239ead..2832c5a348 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt @@ -1,111 +1,101 @@ -WholeStageCodegen (33) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (32) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt index 49645a95cd..ee3242f2b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt @@ -1,74 +1,68 @@ == Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * CometColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (59) +* CometColumnarToRow (64) ++- CometTakeOrderedAndProject (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometSortMergeJoin (58) + :- CometSort (52) + : +- CometExchange (51) + : +- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) + : : +- CometBroadcastExchange (43) + : : +- CometFilter (42) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) + +- CometSort (57) + +- CometExchange (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -83,10 +77,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16, i_item_desc#17] -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [cd_demo_sk#18] -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: [hd_demo_sk#20] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(42) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(43) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(44) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +(45) CometProject +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] +Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(47) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(48) CometBroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: [p_promo_sk#30] -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None +(49) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [1]: [p_promo_sk#30] +Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +(50) CometProject +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] +Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(57) CometColumnarExchange +(51) CometExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(58) CometSort +(52) CometSort Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(54) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(55) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(56) CometExchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +(57) CometSort +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(64) CometSortMergeJoin +(58) CometSortMergeJoin Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +(59) CometProject +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(66) CometHashAggregate +(60) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(61) CometExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +(62) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(63) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(64) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) +BroadcastExchange (69) ++- * CometColumnarToRow (68) + +- CometProject (67) + +- CometFilter (66) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(66) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(67) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(69) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index f97ff81f45..9000d27d1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- 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 39 out of 68 eligible operators (57%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt index 3cdd634e67..678f8c67df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] @@ -8,100 +8,67 @@ WholeStageCodegen (11) CometProject [w_warehouse_name,i_item_desc,d_week_seq] CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometColumnarToRow - InputAdapter - CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometExchange [cs_item_sk,cs_order_number] #2 + CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_date] #11 + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [p_promo_sk] #12 + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 + CometExchange [cr_item_sk,cr_order_number] #13 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt index 49645a95cd..ee3242f2b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt @@ -1,74 +1,68 @@ == Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * CometColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (59) +* CometColumnarToRow (64) ++- CometTakeOrderedAndProject (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometSortMergeJoin (58) + :- CometSort (52) + : +- CometExchange (51) + : +- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) + : : +- CometBroadcastExchange (43) + : : +- CometFilter (42) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) + : +- CometBroadcastExchange (48) + : +- CometFilter (47) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) + +- CometSort (57) + +- CometExchange (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -83,10 +77,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16, i_item_desc#17] -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [cd_demo_sk#18] -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: [hd_demo_sk#20] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(42) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(43) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(44) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +(45) CometProject +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] +Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(47) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(48) CometBroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: [p_promo_sk#30] -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None +(49) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Right output [1]: [p_promo_sk#30] +Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +(50) CometProject +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] +Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(57) CometColumnarExchange +(51) CometExchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(58) CometSort +(52) CometSort Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(54) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(55) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(56) CometExchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] +(57) CometSort +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] -(64) CometSortMergeJoin +(58) CometSortMergeJoin Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter +Right output [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +(59) CometProject +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -(66) CometHashAggregate +(60) CometHashAggregate Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(61) CometExchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] +(62) CometHashAggregate +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(63) CometTakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] +(64) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) +BroadcastExchange (69) ++- * CometColumnarToRow (68) + +- CometProject (67) + +- CometFilter (66) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(66) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] +(67) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(69) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt index f97ff81f45..9000d27d1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- 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 39 out of 68 eligible operators (57%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt index 3cdd634e67..678f8c67df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] @@ -8,100 +8,67 @@ WholeStageCodegen (11) CometProject [w_warehouse_name,i_item_desc,d_week_seq] CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - CometColumnarToRow - InputAdapter - CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] + CometExchange [cs_item_sk,cs_order_number] #2 + CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [i_item_sk,i_item_desc] #6 + CometFilter [i_item_sk,i_item_desc] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange [cd_demo_sk] #7 + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange [hd_demo_sk] #8 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange [d_date_sk,d_week_seq] #10 + CometFilter [d_date_sk,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange [d_date_sk,d_date] #11 + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [p_promo_sk] #12 + CometFilter [p_promo_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 + CometExchange [cr_item_sk,cr_order_number] #13 CometProject [cr_item_sk,cr_order_number] CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt index d3008194f1..b0ae8bb695 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt @@ -1,85 +1,76 @@ == Physical Plan == -* CometColumnarToRow (81) -+- CometTakeOrderedAndProject (80) - +- RowToColumnar (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * CometColumnarToRow (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometFilter (64) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- CometProject (70) + +- CometBroadcastHashJoin (69) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometBroadcastHashJoin (35) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (34) + : : +- CometHashAggregate (33) + : : +- CometExchange (32) + : : +- CometHashAggregate (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (51) + : +- CometFilter (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) + +- CometBroadcastExchange (68) + +- CometHashAggregate (67) + +- CometExchange (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -97,10 +88,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] @@ -108,414 +96,375 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Condition : isnotnull(ss_customer_sk#8) -(7) CometColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] +Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] +(8) CometProject Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -(11) ReusedExchange [Reuses operator id: 85] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] + +(12) CometBroadcastHashJoin +Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] +(13) CometProject Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] +(16) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#15, year_total#16] +Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] +(19) CometFilter +Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] +(20) CometProject +Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(25) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +(22) CometFilter +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) + +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] + +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] +Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight + +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] + +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25, d_year#26] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] +Right output [2]: [d_date_sk#25, d_year#26] +Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] + +(31) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] + +(32) CometExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) +(37) CometFilter +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] +(38) CometProject +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_bill_customer_sk#39) -(44) CometColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] +Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight + +(43) CometProject +Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] + +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#43, d_year#44] + +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight + +(46) CometProject +Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] + +(47) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] + +(48) CometExchange +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(49) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] + +(50) CometFilter +Input [2]: [customer_id#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#46, year_total#47] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Right output [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] +Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 85] -Output [2]: [d_date_sk#46, d_year#47] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#48] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#50] -Results [2]: [c_customer_id#39 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#50,17,2) AS year_total#52] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#51, year_total#52] - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(60) CometFilter -Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true))) +(55) CometFilter +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) -(61) CometProject -Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -Arguments: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#55, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#56, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41] +(56) CometProject +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] -Condition : isnotnull(ws_bill_customer_sk#57) - -(65) CometColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#53] -Right keys [1]: [ws_bill_customer_sk#57] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59] -Input [7]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] - -(69) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#61, d_year#62] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59, d_date_sk#61, d_year#62] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum#63] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] -Functions [1]: [sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#58))#50] -Results [2]: [c_customer_id#39 AS customer_id#65, MakeDecimal(sum(UnscaledValue(ws_net_paid#58))#50,17,2) AS year_total#66] +(58) CometFilter +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#52) -(76) BroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] +Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight + +(61) CometProject +Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#56, d_year#57] -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#65] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#66 / year_total#52) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) +(63) CometBroadcastHashJoin +Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] +Right output [2]: [d_date_sk#56, d_year#57] +Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52, customer_id#65, year_total#66] +(64) CometProject +Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -(79) RowToColumnar -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +(65) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] -(80) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] +(66) CometExchange +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(81) CometColumnarToRow [codegen id : 17] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +(67) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] +Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#59, year_total#60] +Arguments: [customer_id#59, year_total#60] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Right output [2]: [customer_id#59, year_total#60] +Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] +Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] + +(71) CometTakeOrderedAndProject +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] + +(72) CometColumnarToRow [codegen id : 1] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometFilter (83) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(83) CometFilter +(74) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(85) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) +(78) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(88) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#25, d_year#26] -(89) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(80) BroadcastExchange +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index f5dda0de66..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -1,107 +1,90 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt index cc6790a5db..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt @@ -1,134 +1,90 @@ -WholeStageCodegen (17) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - RowToColumnar - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt index d3008194f1..b0ae8bb695 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt @@ -1,85 +1,76 @@ == Physical Plan == -* CometColumnarToRow (81) -+- CometTakeOrderedAndProject (80) - +- RowToColumnar (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * CometColumnarToRow (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * CometColumnarToRow (65) - : +- CometFilter (64) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) +* CometColumnarToRow (72) ++- CometTakeOrderedAndProject (71) + +- CometProject (70) + +- CometBroadcastHashJoin (69) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometBroadcastHashJoin (35) + : : :- CometFilter (17) + : : : +- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (34) + : : +- CometHashAggregate (33) + : : +- CometExchange (32) + : : +- CometHashAggregate (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) + : +- CometBroadcastExchange (51) + : +- CometFilter (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (38) + : : : +- CometFilter (37) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) + +- CometBroadcastExchange (68) + +- CometHashAggregate (67) + +- CometExchange (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -97,10 +88,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] @@ -108,414 +96,375 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Condition : isnotnull(ss_customer_sk#8) -(7) CometColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] +Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] +(8) CometProject Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -(11) ReusedExchange [Reuses operator id: 85] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] + +(12) CometBroadcastHashJoin +Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] +(13) CometProject Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] +(16) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) +(17) CometFilter +Input [2]: [customer_id#15, year_total#16] +Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] +(19) CometFilter +Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] +(20) CometProject +Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] +Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(25) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] +(22) CometFilter +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) + +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] + +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] +Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight + +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] + +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(27) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25, d_year#26] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] +Right output [2]: [d_date_sk#25, d_year#26] +Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] + +(31) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] + +(32) CometExchange +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] +Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] +Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) +(37) CometFilter +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] +(38) CometProject +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_bill_customer_sk#39) -(44) CometColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] +Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight + +(43) CometProject +Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] + +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#43, d_year#44] + +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] +Right output [2]: [d_date_sk#43, d_year#44] +Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight + +(46) CometProject +Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] + +(47) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] + +(48) CometExchange +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(49) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] + +(50) CometFilter +Input [2]: [customer_id#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#46, year_total#47] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Right output [2]: [customer_id#46, year_total#47] +Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] +Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 85] -Output [2]: [d_date_sk#46, d_year#47] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#48] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#49] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#50] -Results [2]: [c_customer_id#39 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#50,17,2) AS year_total#52] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#51, year_total#52] - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(60) CometFilter -Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true))) +(55) CometFilter +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) -(61) CometProject -Input [4]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56] -Arguments: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#54, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#55, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#56, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41] +(56) CometProject +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#59), dynamicpruningexpression(ws_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] -Condition : isnotnull(ws_bill_customer_sk#57) - -(65) CometColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#53] -Right keys [1]: [ws_bill_customer_sk#57] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59] -Input [7]: [c_customer_sk#53, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#57, ws_net_paid#58, ws_sold_date_sk#59] - -(69) ReusedExchange [Reuses operator id: 89] -Output [2]: [d_date_sk#61, d_year#62] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, ws_sold_date_sk#59, d_date_sk#61, d_year#62] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#58, d_year#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum#63] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62, sum#64] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#62] -Functions [1]: [sum(UnscaledValue(ws_net_paid#58))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#58))#50] -Results [2]: [c_customer_id#39 AS customer_id#65, MakeDecimal(sum(UnscaledValue(ws_net_paid#58))#50,17,2) AS year_total#66] +(58) CometFilter +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#52) -(76) BroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] +Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight + +(61) CometProject +Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#56, d_year#57] -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#65] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#66 / year_total#52) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) +(63) CometBroadcastHashJoin +Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] +Right output [2]: [d_date_sk#56, d_year#57] +Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#52, customer_id#65, year_total#66] +(64) CometProject +Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] +Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -(79) RowToColumnar -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +(65) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] -(80) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] +(66) CometExchange +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(81) CometColumnarToRow [codegen id : 17] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +(67) CometHashAggregate +Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] +Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] +Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#59, year_total#60] +Arguments: [customer_id#59, year_total#60] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] +Right output [2]: [customer_id#59, year_total#60] +Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] +Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] + +(71) CometTakeOrderedAndProject +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] + +(72) CometColumnarToRow [codegen id : 1] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (85) -+- * CometColumnarToRow (84) - +- CometFilter (83) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (82) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(83) CometFilter +(74) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(84) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(85) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) +(78) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(88) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#25, d_year#26] -(89) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(80) BroadcastExchange +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#59 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt index f5dda0de66..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt @@ -1,107 +1,90 @@ CometColumnarToRow +- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 37 out of 85 eligible operators (43%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt index cc6790a5db..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt @@ -1,134 +1,90 @@ -WholeStageCodegen (17) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - RowToColumnar - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometProject [customer_id,customer_first_name,customer_last_name] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #4 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 + CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [customer_id,year_total] #10 + CometFilter [customer_id,year_total] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 + CometBroadcastExchange [customer_id,year_total] #13 + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] + CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15 + CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt index b48350e78c..6925eb301e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt @@ -1,131 +1,128 @@ == Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) +* CometColumnarToRow (124) ++- CometTakeOrderedAndProject (123) + +- CometProject (122) + +- CometSortMergeJoin (121) + :- CometSort (66) + : +- CometExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) : :- CometProject (22) : : +- CometSortMergeJoin (21) : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) : : +- CometSort (20) : : +- CometExchange (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * CometColumnarToRow (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * CometColumnarToRow (43) - : : : : +- CometFilter (42) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * CometColumnarToRow (71) - : : : : +- CometFilter (70) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * CometColumnarToRow (86) - : : : : +- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * CometColumnarToRow (101) - : : : +- CometFilter (100) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) + +- CometSort (120) + +- CometExchange (119) + +- CometFilter (118) + +- CometHashAggregate (117) + +- CometExchange (116) + +- CometHashAggregate (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometProject (82) + : +- CometSortMergeJoin (81) + : :- CometSort (78) + : : +- CometExchange (77) + : : +- CometProject (76) + : : +- CometBroadcastHashJoin (75) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometFilter (68) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) + : : : +- ReusedExchange (69) + : : +- CometBroadcastExchange (74) + : : +- CometFilter (73) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) + : +- CometSort (80) + : +- ReusedExchange (79) + :- CometProject (96) + : +- CometSortMergeJoin (95) + : :- CometSort (92) + : : +- CometExchange (91) + : : +- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometFilter (84) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (88) + : +- CometSort (94) + : +- ReusedExchange (93) + +- CometProject (110) + +- CometSortMergeJoin (109) + :- CometSort (106) + : +- CometExchange (105) + : +- CometProject (104) + : +- CometBroadcastHashJoin (103) + : :- CometProject (101) + : : +- CometBroadcastHashJoin (100) + : : :- CometFilter (98) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) + : : +- ReusedExchange (99) + : +- ReusedExchange (102) + +- CometSort (108) + +- ReusedExchange (107) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -140,57 +137,61 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 131] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) CometColumnarExchange +(14) CometExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] (15) CometSort Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] @@ -213,7 +214,7 @@ Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_ (19) CometExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (20) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -240,76 +241,71 @@ ReadSchema: struct -(35) CometFilter +(34) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(36) CometProject +(35) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(37) CometExchange +(36) CometExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(38) CometSort +(37) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(39) CometSortMergeJoin +(38) CometSortMergeJoin Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(40) CometProject +(39) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -317,125 +313,120 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) CometFilter +(41) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(43) CometColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(44) ReusedExchange [Reuses operator id: 8] +(42) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(43) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(44) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) ReusedExchange [Reuses operator id: 131] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(46) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(47) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(50) CometColumnarExchange +(48) CometExchange Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(51) CometSort +(49) CometSort Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(53) CometFilter +(51) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(54) CometProject +(52) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(55) CometExchange +(53) CometExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(56) CometSort +(54) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(57) CometSortMergeJoin +(55) CometSortMergeJoin Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(58) CometProject +(56) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(59) CometUnion +(57) CometUnion Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(60) CometHashAggregate +(58) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) CometExchange +(59) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(62) CometHashAggregate +(60) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(63) CometHashAggregate +(61) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(64) CometExchange +(62) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(65) CometHashAggregate +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(66) CometFilter +(64) CometFilter Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Condition : isnotnull(sales_cnt#64) -(67) CometExchange +(65) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(68) CometSort +(66) CometSort Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] @@ -443,64 +434,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter +(68) CometFilter Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Condition : isnotnull(cs_item_sk#66) -(71) CometColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] - -(72) ReusedExchange [Reuses operator id: 8] +(69) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#66] -Right keys [1]: [i_item_sk#72] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(71) CometProject Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(75) ReusedExchange [Reuses operator id: 135] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#70] -Right keys [1]: [d_date_sk#77] -Join type: Inner -Join condition: None +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] + +(75) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +(76) CometProject Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(78) CometColumnarExchange +(77) CometExchange Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(79) CometSort +(78) CometSort Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(80) ReusedExchange [Reuses operator id: 19] +(79) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(81) CometSort +(80) CometSort Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(82) CometSortMergeJoin +(81) CometSortMergeJoin Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(83) CometProject +(82) CometProject Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -508,64 +506,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter +(84) CometFilter Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) -(86) CometColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] - -(87) ReusedExchange [Reuses operator id: 8] +(85) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#89] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(87) CometProject Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(90) ReusedExchange [Reuses operator id: 135] +(88) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#94, d_year#95] -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#94] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +(90) CometProject Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(93) CometColumnarExchange +(91) CometExchange Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(94) CometSort +(92) CometSort Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(95) ReusedExchange [Reuses operator id: 37] +(93) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(96) CometSort +(94) CometSort Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(97) CometSortMergeJoin +(95) CometSortMergeJoin Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(98) CometProject +(96) CometProject Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] @@ -573,182 +566,177 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) CometFilter +(98) CometFilter Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Condition : isnotnull(ws_item_sk#100) -(101) CometColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] - -(102) ReusedExchange [Reuses operator id: 8] +(99) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#100] -Right keys [1]: [i_item_sk#106] -Join type: Inner -Join condition: None +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(101) CometProject Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(105) ReusedExchange [Reuses operator id: 135] +(102) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#111, d_year#112] -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#104] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +(104) CometProject Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(108) CometColumnarExchange +(105) CometExchange Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(109) CometSort +(106) CometSort Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(110) ReusedExchange [Reuses operator id: 55] +(107) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(111) CometSort +(108) CometSort Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(112) CometSortMergeJoin +(109) CometSortMergeJoin Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(113) CometProject +(110) CometProject Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(114) CometUnion +(111) CometUnion Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(115) CometHashAggregate +(112) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(116) CometExchange +(113) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(117) CometHashAggregate +(114) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(118) CometHashAggregate +(115) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(119) CometExchange +(116) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] -(120) CometHashAggregate +(117) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(121) CometFilter +(118) CometFilter Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Condition : isnotnull(sales_cnt#118) -(122) CometExchange +(119) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(123) CometSort +(120) CometSort Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(124) CometSortMergeJoin +(121) CometSortMergeJoin Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(125) CometProject +(122) CometProject Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -(126) CometTakeOrderedAndProject +(123) CometTakeOrderedAndProject Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(127) CometColumnarToRow [codegen id : 19] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(129) CometFilter +(126) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(131) BroadcastExchange +(128) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (132) +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (132) ++- * CometColumnarToRow (131) + +- CometFilter (130) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) -(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(133) CometFilter +(130) CometFilter Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(134) CometColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] -(135) BroadcastExchange +(132) BroadcastExchange Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 0765ce2338..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 @@ -45,24 +42,21 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- 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 @@ -71,24 +65,21 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- 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 @@ -107,28 +98,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 @@ -137,24 +125,21 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- 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 @@ -163,28 +148,25 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- 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 123 out of 167 eligible operators (73%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt index 2f283a3eec..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -17,157 +17,128 @@ WholeStageCodegen (19) CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [cs_order_number,cs_item_sk] #4 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 + CometExchange [cr_order_number,cr_item_sk] #8 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [ss_ticket_number,ss_item_sk] #9 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 + CometExchange [sr_ticket_number,sr_item_sk] #10 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [ws_order_number,ws_item_sk] #11 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 + CometExchange [wr_order_number,wr_item_sk] #12 CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [cs_order_number,cs_item_sk] #16 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [ss_ticket_number,ss_item_sk] #19 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [ws_order_number,ws_item_sk] #20 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt index b48350e78c..6925eb301e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt @@ -1,131 +1,128 @@ == Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) +* CometColumnarToRow (124) ++- CometTakeOrderedAndProject (123) + +- CometProject (122) + +- CometSortMergeJoin (121) + :- CometSort (66) + : +- CometExchange (65) + : +- CometFilter (64) + : +- CometHashAggregate (63) + : +- CometExchange (62) + : +- CometHashAggregate (61) + : +- CometHashAggregate (60) + : +- CometExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) : :- CometProject (22) : : +- CometSortMergeJoin (21) : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) : : +- CometSort (20) : : +- CometExchange (19) : : +- CometProject (18) : : +- CometFilter (17) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * CometColumnarToRow (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * CometColumnarToRow (43) - : : : : +- CometFilter (42) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * CometColumnarToRow (71) - : : : : +- CometFilter (70) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * CometColumnarToRow (86) - : : : : +- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * CometColumnarToRow (101) - : : : +- CometFilter (100) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometExchange (31) + : : : +- CometProject (30) + : : : +- CometBroadcastHashJoin (29) + : : : :- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) + : : : : +- ReusedExchange (25) + : : : +- ReusedExchange (28) + : : +- CometSort (37) + : : +- CometExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (48) + : : +- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometFilter (41) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) + : : : +- ReusedExchange (42) + : : +- ReusedExchange (45) + : +- CometSort (54) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) + +- CometSort (120) + +- CometExchange (119) + +- CometFilter (118) + +- CometHashAggregate (117) + +- CometExchange (116) + +- CometHashAggregate (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometProject (82) + : +- CometSortMergeJoin (81) + : :- CometSort (78) + : : +- CometExchange (77) + : : +- CometProject (76) + : : +- CometBroadcastHashJoin (75) + : : :- CometProject (71) + : : : +- CometBroadcastHashJoin (70) + : : : :- CometFilter (68) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) + : : : +- ReusedExchange (69) + : : +- CometBroadcastExchange (74) + : : +- CometFilter (73) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) + : +- CometSort (80) + : +- ReusedExchange (79) + :- CometProject (96) + : +- CometSortMergeJoin (95) + : :- CometSort (92) + : : +- CometExchange (91) + : : +- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometFilter (84) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (88) + : +- CometSort (94) + : +- ReusedExchange (93) + +- CometProject (110) + +- CometSortMergeJoin (109) + :- CometSort (106) + : +- CometExchange (105) + : +- CometProject (104) + : +- CometBroadcastHashJoin (103) + : :- CometProject (101) + : : +- CometBroadcastHashJoin (100) + : : :- CometFilter (98) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) + : : +- ReusedExchange (99) + : +- ReusedExchange (102) + +- CometSort (108) + +- ReusedExchange (107) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -140,57 +137,61 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 131] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) CometColumnarExchange +(14) CometExchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] (15) CometSort Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] @@ -213,7 +214,7 @@ Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_ (19) CometExchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (20) CometSort Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] @@ -240,76 +241,71 @@ ReadSchema: struct -(35) CometFilter +(34) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(36) CometProject +(35) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(37) CometExchange +(36) CometExchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(38) CometSort +(37) CometSort Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] -(39) CometSortMergeJoin +(38) CometSortMergeJoin Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter -(40) CometProject +(39) CometProject Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -317,125 +313,120 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) CometFilter +(41) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(43) CometColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(44) ReusedExchange [Reuses operator id: 8] +(42) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(43) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(44) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) ReusedExchange [Reuses operator id: 131] +(45) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(46) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(47) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(50) CometColumnarExchange +(48) CometExchange Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(51) CometSort +(49) CometSort Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(53) CometFilter +(51) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(54) CometProject +(52) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(55) CometExchange +(53) CometExchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(56) CometSort +(54) CometSort Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] -(57) CometSortMergeJoin +(55) CometSortMergeJoin Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter -(58) CometProject +(56) CometProject Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] -(59) CometUnion +(57) CometUnion Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] -(60) CometHashAggregate +(58) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(61) CometExchange +(59) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(62) CometHashAggregate +(60) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] -(63) CometHashAggregate +(61) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(64) CometExchange +(62) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(65) CometHashAggregate +(63) CometHashAggregate Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(66) CometFilter +(64) CometFilter Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Condition : isnotnull(sales_cnt#64) -(67) CometExchange +(65) CometExchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(68) CometSort +(66) CometSort Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Batched: true Location: InMemoryFileIndex [] @@ -443,64 +434,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter +(68) CometFilter Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] Condition : isnotnull(cs_item_sk#66) -(71) CometColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] - -(72) ReusedExchange [Reuses operator id: 8] +(69) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#66] -Right keys [1]: [i_item_sk#72] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] +Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +(71) CometProject Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -(75) ReusedExchange [Reuses operator id: 135] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(73) CometFilter +Input [2]: [d_date_sk#77, d_year#78] +Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#70] -Right keys [1]: [d_date_sk#77] -Join type: Inner -Join condition: None +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] + +(75) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] +Right output [2]: [d_date_sk#77, d_year#78] +Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] +(76) CometProject Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] +Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -(78) CometColumnarExchange +(77) CometExchange Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] -(79) CometSort +(78) CometSort Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] -(80) ReusedExchange [Reuses operator id: 19] +(79) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -(81) CometSort +(80) CometSort Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] -(82) CometSortMergeJoin +(81) CometSortMergeJoin Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter -(83) CometProject +(82) CometProject Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] @@ -508,64 +506,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter +(84) CometFilter Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) -(86) CometColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] - -(87) ReusedExchange [Reuses operator id: 8] +(85) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#89] -Join type: Inner -Join condition: None +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] +Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +(87) CometProject Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -(90) ReusedExchange [Reuses operator id: 135] +(88) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#94, d_year#95] -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#94] -Join type: Inner -Join condition: None +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] +Right output [2]: [d_date_sk#94, d_year#95] +Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] +(90) CometProject Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] +Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -(93) CometColumnarExchange +(91) CometExchange Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(94) CometSort +(92) CometSort Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] -(95) ReusedExchange [Reuses operator id: 37] +(93) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -(96) CometSort +(94) CometSort Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] -(97) CometSortMergeJoin +(95) CometSortMergeJoin Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter -(98) CometProject +(96) CometProject Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Batched: true Location: InMemoryFileIndex [] @@ -573,182 +566,177 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) CometFilter +(98) CometFilter Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] Condition : isnotnull(ws_item_sk#100) -(101) CometColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] - -(102) ReusedExchange [Reuses operator id: 8] +(99) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#100] -Right keys [1]: [i_item_sk#106] -Join type: Inner -Join condition: None +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] +Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +(101) CometProject Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(105) ReusedExchange [Reuses operator id: 135] +(102) ReusedExchange [Reuses operator id: 74] Output [2]: [d_date_sk#111, d_year#112] -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#104] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] +Right output [2]: [d_date_sk#111, d_year#112] +Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] +(104) CometProject Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] +Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -(108) CometColumnarExchange +(105) CometExchange Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(109) CometSort +(106) CometSort Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] -(110) ReusedExchange [Reuses operator id: 55] +(107) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -(111) CometSort +(108) CometSort Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] -(112) CometSortMergeJoin +(109) CometSortMergeJoin Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter -(113) CometProject +(110) CometProject Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] -(114) CometUnion +(111) CometUnion Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] -(115) CometHashAggregate +(112) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(116) CometExchange +(113) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(117) CometHashAggregate +(114) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Functions: [] -(118) CometHashAggregate +(115) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] -(119) CometExchange +(116) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] -(120) CometHashAggregate +(117) CometHashAggregate Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] -(121) CometFilter +(118) CometFilter Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Condition : isnotnull(sales_cnt#118) -(122) CometExchange +(119) CometExchange Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(123) CometSort +(120) CometSort Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] -(124) CometSortMergeJoin +(121) CometSortMergeJoin Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) -(125) CometProject +(122) CometProject Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] -(126) CometTakeOrderedAndProject +(123) CometTakeOrderedAndProject Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -(127) CometColumnarToRow [codegen id : 19] +(124) CometColumnarToRow [codegen id : 1] Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(129) CometFilter +(126) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(131) BroadcastExchange +(128) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (132) +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (132) ++- * CometColumnarToRow (131) + +- CometFilter (130) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) -(132) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(133) CometFilter +(130) CometFilter Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(134) CometColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] -(135) BroadcastExchange +(132) BroadcastExchange Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt index 0765ce2338..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 @@ -45,24 +42,21 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- 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 @@ -71,24 +65,21 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- 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 @@ -107,28 +98,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 @@ -137,24 +125,21 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- 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 @@ -163,28 +148,25 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- 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 123 out of 167 eligible operators (73%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt index 2f283a3eec..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -17,157 +17,128 @@ WholeStageCodegen (19) CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [cs_order_number,cs_item_sk] #4 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange [d_date_sk,d_year] #7 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 + CometExchange [cr_order_number,cr_item_sk] #8 CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [ss_ticket_number,ss_item_sk] #9 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 + CometExchange [sr_ticket_number,sr_item_sk] #10 CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometExchange [ws_order_number,ws_item_sk] #11 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 + CometExchange [wr_order_number,wr_item_sk] #12 CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [cs_order_number,cs_item_sk] #16 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #17 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange [d_date_sk,d_year] #18 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [ss_ticket_number,ss_item_sk] #19 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + CometExchange [ws_order_number,ws_item_sk] #20 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt index 5f6ee267fb..4b5efc84e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt @@ -1,114 +1,107 @@ == Physical Plan == -* CometColumnarToRow (110) -+- CometTakeOrderedAndProject (109) - +- CometHashAggregate (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * HashAggregate (83) + : +- * CometColumnarToRow (82) + : +- CometColumnarExchange (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : : +- CometBroadcastExchange (28) + : : +- CometHashAggregate (27) + : : +- CometExchange (26) + : : +- CometHashAggregate (25) + : : +- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (21) + : : : +- CometBroadcastHashJoin (20) + : : : :- CometFilter (18) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (50) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : : :- BroadcastExchange (40) + : : : +- * CometColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * CometColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * CometColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometFilter (52) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) + : +- CometBroadcastExchange (75) + : +- CometHashAggregate (74) + : +- CometExchange (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) :- * HashAggregate (90) : +- * CometColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) - : +- Union (86) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometColumnarExchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * CometColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (53) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : : :- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) - : : : +- CometColumnarExchange (39) - : : : +- * HashAggregate (38) - : : : +- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * CometColumnarToRow (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (35) - : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) - : : +- CometColumnarExchange (49) - : : +- * HashAggregate (48) - : : +- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : +- ReusedExchange (45) - : +- * Project (85) - : +- * BroadcastHashJoin LeftOuter BuildRight (84) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * CometColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (60) - : +- BroadcastExchange (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- CometColumnarExchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * CometColumnarToRow (72) - : : : +- CometFilter (71) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (70) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - :- * HashAggregate (97) - : +- * CometColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- * HashAggregate (94) - : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) - : +- ReusedExchange (91) - +- * HashAggregate (104) - +- * CometColumnarToRow (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- * HashAggregate (100) - +- * CometColumnarToRow (99) - +- ReusedExchange (98) + : +- * HashAggregate (86) + : +- * CometColumnarToRow (85) + : +- ReusedExchange (84) + +- * HashAggregate (97) + +- * CometColumnarToRow (96) + +- CometColumnarExchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- * CometColumnarToRow (92) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -123,556 +116,506 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(14) CometHashAggregate +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(16) CometHashAggregate +Input [3]: [s_store_sk#8, sum#9, sum#10] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (18) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) -(19) CometColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(20) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#21] +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(30) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] -Join type: LeftOuter -Join condition: None +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(32) Project [codegen id : 8] -Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(31) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) CometColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#40] +(34) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(35) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] + +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] + +(37) CometExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] + +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] + +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(44) CometColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(45) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#53] +(43) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(44) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] - -(49) CometColumnarExchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(50) CometColumnarToRow -Input [2]: [sum#56, sum#57] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(51) HashAggregate -Input [2]: [sum#56, sum#57] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(50) Project [codegen id : 3] +Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) - -(56) CometColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(57) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#70] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(61) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] + +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] + +(62) CometExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] + +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) - -(72) CometColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] - -(73) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#85] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#86] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] - -(86) Union - -(87) HashAggregate [codegen id : 23] -Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] -Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) + +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] + +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight + +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] + +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] + +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight + +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] + +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(74) CometHashAggregate +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] + +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] + +(78) CometColumnarToRow [codegen id : 4] +Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] + +(79) Union + +(80) HashAggregate [codegen id : 5] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +Keys [2]: [channel#24, id#25] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] +Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(81) CometColumnarExchange +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(82) CometColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(83) HashAggregate [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] + +(84) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) HashAggregate [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] + +(87) HashAggregate [codegen id : 12] +Input [4]: [channel#24, sales#94, returns#95, profit#96] +Keys [1]: [channel#24] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] (88) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(89) CometColumnarToRow [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(90) HashAggregate [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] - -(91) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(92) CometColumnarToRow [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(93) HashAggregate [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(94) HashAggregate [codegen id : 48] -Input [4]: [channel#31, sales#117, returns#118, profit#119] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(95) CometColumnarExchange -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometColumnarToRow [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(97) HashAggregate [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] -Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] - -(98) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(99) CometColumnarToRow [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(100) HashAggregate [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [3]: [sum(sales#14)#111 AS sales#139, sum(returns#33)#112 AS returns#140, sum(profit#34)#113 AS profit#141] - -(101) HashAggregate [codegen id : 73] -Input [3]: [sales#139, returns#140, profit#141] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) HashAggregate [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [1]: [channel#24] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] +Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] +Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] + +(91) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(93) HashAggregate [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] + +(94) HashAggregate [codegen id : 19] +Input [3]: [sales#116, returns#117, profit#118] Keys: [] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -(102) CometColumnarExchange -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(95) CometColumnarExchange +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(103) CometColumnarToRow [codegen id : 74] -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +(96) CometColumnarToRow [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -(104) HashAggregate [codegen id : 74] -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +(97) HashAggregate [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys: [] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [null AS channel#157, null AS id#158, sum(sales#139)#154 AS sales#159, sum(returns#140)#155 AS returns#160, sum(profit#141)#156 AS profit#161] +Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] +Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] +Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] -(105) Union +(98) Union -(106) HashAggregate [codegen id : 75] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(99) HashAggregate [codegen id : 21] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(107) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(100) CometColumnarExchange +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(108) CometHashAggregate -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(101) CometHashAggregate +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] -(109) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), [channel#31, id#32, sales#114, returns#115, profit#116], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] +(102) CometTakeOrderedAndProject +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] -(110) CometColumnarToRow [codegen id : 76] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(103) CometColumnarToRow [codegen id : 22] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (111) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#162] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(112) CometFilter -Input [2]: [d_date_sk#6, d_date#162] -Condition : (((isnotnull(d_date#162) AND (d_date#162 >= 1998-08-04)) AND (d_date#162 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(105) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(113) CometProject -Input [2]: [d_date_sk#6, d_date#162] +(106) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(114) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(115) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index 9419932935..3b44c6c27c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -9,132 +9,114 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 : +- CometColumnarToRow : +- CometColumnarExchange @@ -144,132 +126,114 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 +- CometColumnarToRow +- CometColumnarExchange @@ -279,131 +243,113 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 + +- CometColumnarToRow + +- 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 143 out of 332 eligible operators (43%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt index c67ddd5c51..ac3d312ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt @@ -1,178 +1,139 @@ -WholeStageCodegen (76) +WholeStageCodegen (22) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (21) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (6) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) + WholeStageCodegen (5) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometExchange [s_store_sk] #3 + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometExchange [s_store_sk] #8 + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 + WholeStageCodegen (3) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + BroadcastExchange #9 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [cs_call_center_sk] #10 + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) + ReusedExchange [d_date_sk] #5 + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #11 + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometExchange [wp_web_page_sk] #12 + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [wp_web_page_sk] #13 + CometFilter [wp_web_page_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometExchange [wp_web_page_sk] #15 + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + ReusedExchange [wp_web_page_sk] #13 + WholeStageCodegen (13) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) + CometColumnarExchange [channel] #16 + WholeStageCodegen (12) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + WholeStageCodegen (20) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) + CometColumnarExchange #17 + WholeStageCodegen (19) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt index 5f6ee267fb..4b5efc84e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt @@ -1,114 +1,107 @@ == Physical Plan == -* CometColumnarToRow (110) -+- CometTakeOrderedAndProject (109) - +- CometHashAggregate (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * HashAggregate (83) + : +- * CometColumnarToRow (82) + : +- CometColumnarExchange (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometExchange (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : : +- CometBroadcastExchange (28) + : : +- CometHashAggregate (27) + : : +- CometExchange (26) + : : +- CometHashAggregate (25) + : : +- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (21) + : : : +- CometBroadcastHashJoin (20) + : : : :- CometFilter (18) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (50) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) + : : :- BroadcastExchange (40) + : : : +- * CometColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * CometColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * CometColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometExchange (62) + : : +- CometHashAggregate (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometFilter (52) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) + : : : +- ReusedExchange (53) + : : +- CometBroadcastExchange (58) + : : +- CometFilter (57) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) + : +- CometBroadcastExchange (75) + : +- CometHashAggregate (74) + : +- CometExchange (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) :- * HashAggregate (90) : +- * CometColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) - : +- Union (86) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometColumnarExchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * CometColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (53) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : : :- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) - : : : +- CometColumnarExchange (39) - : : : +- * HashAggregate (38) - : : : +- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * CometColumnarToRow (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (35) - : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) - : : +- CometColumnarExchange (49) - : : +- * HashAggregate (48) - : : +- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : +- ReusedExchange (45) - : +- * Project (85) - : +- * BroadcastHashJoin LeftOuter BuildRight (84) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * CometColumnarToRow (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (60) - : +- BroadcastExchange (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- CometColumnarExchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * CometColumnarToRow (72) - : : : +- CometFilter (71) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (70) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - :- * HashAggregate (97) - : +- * CometColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- * HashAggregate (94) - : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) - : +- ReusedExchange (91) - +- * HashAggregate (104) - +- * CometColumnarToRow (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- * HashAggregate (100) - +- * CometColumnarToRow (99) - +- ReusedExchange (98) + : +- * HashAggregate (86) + : +- * CometColumnarToRow (85) + : +- ReusedExchange (84) + +- * HashAggregate (97) + +- * CometColumnarToRow (96) + +- CometColumnarExchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- * CometColumnarToRow (92) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -123,556 +116,506 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(14) CometHashAggregate +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(15) CometExchange +Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(16) CometHashAggregate +Input [3]: [s_store_sk#8, sum#9, sum#10] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] (17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (18) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Condition : isnotnull(sr_store_sk#11) -(19) CometColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(19) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(20) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#21] +(20) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] +Right output [1]: [d_date_sk#16] +Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(21) CometProject +Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] +Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(22) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#17] -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(23) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] +Right output [1]: [s_store_sk#17] +Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(24) CometProject +Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(30) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] -Join type: LeftOuter -Join condition: None +(25) CometHashAggregate +Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] +Keys [1]: [s_store_sk#17] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] + +(26) CometExchange +Input [3]: [s_store_sk#17, sum#18, sum#19] +Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(27) CometHashAggregate +Input [3]: [s_store_sk#17, sum#18, sum#19] +Keys [1]: [s_store_sk#17] +Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] + +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] + +(29) CometBroadcastHashJoin +Left output [3]: [s_store_sk#8, sales#22, profit#23] +Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight -(32) Project [codegen id : 8] -Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(31) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] + +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) CometColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#40] +(34) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(35) CometProject +Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] +Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] + +(36) CometHashAggregate +Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] + +(37) CometExchange +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(38) CometHashAggregate +Input [3]: [cs_call_center_sk#28, sum#34, sum#35] +Keys [1]: [cs_call_center_sk#28] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] + +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] + +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(44) CometColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(45) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#53] +(43) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Right output [1]: [d_date_sk#42] +Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(44) CometProject +Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] +Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] - -(49) CometColumnarExchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(50) CometColumnarToRow -Input [2]: [sum#56, sum#57] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(51) HashAggregate -Input [2]: [sum#56, sum#57] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(50) Project [codegen id : 3] +Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) - -(56) CometColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) CometFilter +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_web_page_sk#50) -(57) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#70] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] +Right output [1]: [d_date_sk#55] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(55) CometProject +Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] +Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(61) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(59) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] +Right output [1]: [wp_web_page_sk#56] +Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None +(60) CometProject +Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] + +(61) CometHashAggregate +Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] + +(62) CometExchange +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(63) CometHashAggregate +Input [3]: [wp_web_page_sk#56, sum#57, sum#58] +Keys [1]: [wp_web_page_sk#56] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] + +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) - -(72) CometColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] - -(73) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#85] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#86] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] - -(86) Union - -(87) HashAggregate [codegen id : 23] -Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] -Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +(65) CometFilter +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Condition : isnotnull(wr_web_page_sk#59) + +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] + +(67) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] +Right output [1]: [d_date_sk#64] +Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight + +(68) CometProject +Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] +Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] + +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] + +(70) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] +Right output [1]: [wp_web_page_sk#65] +Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight + +(71) CometProject +Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] + +(72) CometHashAggregate +Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] + +(73) CometExchange +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(74) CometHashAggregate +Input [3]: [wp_web_page_sk#65, sum#66, sum#67] +Keys [1]: [wp_web_page_sk#65] +Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] + +(75) CometBroadcastExchange +Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] + +(76) CometBroadcastHashJoin +Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] +Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight + +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] + +(78) CometColumnarToRow [codegen id : 4] +Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] + +(79) Union + +(80) HashAggregate [codegen id : 5] +Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] +Keys [2]: [channel#24, id#25] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] +Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] +Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(81) CometColumnarExchange +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(82) CometColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(83) HashAggregate [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] + +(84) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) HashAggregate [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] + +(87) HashAggregate [codegen id : 12] +Input [4]: [channel#24, sales#94, returns#95, profit#96] +Keys [1]: [channel#24] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] +Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] (88) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(89) CometColumnarToRow [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(90) HashAggregate [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] - -(91) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(92) CometColumnarToRow [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(93) HashAggregate [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(94) HashAggregate [codegen id : 48] -Input [4]: [channel#31, sales#117, returns#118, profit#119] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(95) CometColumnarExchange -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometColumnarToRow [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(97) HashAggregate [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] -Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] - -(98) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(99) CometColumnarToRow [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(100) HashAggregate [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [3]: [sum(sales#14)#111 AS sales#139, sum(returns#33)#112 AS returns#140, sum(profit#34)#113 AS profit#141] - -(101) HashAggregate [codegen id : 73] -Input [3]: [sales#139, returns#140, profit#141] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) HashAggregate [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Keys [1]: [channel#24] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] +Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] +Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] + +(91) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(93) HashAggregate [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] +Keys [2]: [channel#24, id#25] +Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] +Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] +Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] + +(94) HashAggregate [codegen id : 19] +Input [3]: [sales#116, returns#117, profit#118] Keys: [] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -(102) CometColumnarExchange -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(95) CometColumnarExchange +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(103) CometColumnarToRow [codegen id : 74] -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +(96) CometColumnarToRow [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -(104) HashAggregate [codegen id : 74] -Input [6]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] +(97) HashAggregate [codegen id : 20] +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys: [] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [null AS channel#157, null AS id#158, sum(sales#139)#154 AS sales#159, sum(returns#140)#155 AS returns#160, sum(profit#141)#156 AS profit#161] +Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] +Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] +Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] -(105) Union +(98) Union -(106) HashAggregate [codegen id : 75] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(99) HashAggregate [codegen id : 21] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(107) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(100) CometColumnarExchange +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(108) CometHashAggregate -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(101) CometHashAggregate +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Functions: [] -(109) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#114,returns#115,profit#116]), [channel#31, id#32, sales#114, returns#115, profit#116], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] +(102) CometTakeOrderedAndProject +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] -(110) CometColumnarToRow [codegen id : 76] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(103) CometColumnarToRow [codegen id : 22] +Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (111) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#162] +(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(112) CometFilter -Input [2]: [d_date_sk#6, d_date#162] -Condition : (((isnotnull(d_date#162) AND (d_date#162 >= 1998-08-04)) AND (d_date#162 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(105) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(113) CometProject -Input [2]: [d_date_sk#6, d_date#162] +(106) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(114) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(115) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt index 9419932935..3b44c6c27c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt @@ -9,132 +9,114 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 : +- CometColumnarToRow : +- CometColumnarExchange @@ -144,132 +126,114 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- 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 : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 +- CometColumnarToRow +- CometColumnarExchange @@ -279,131 +243,113 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- 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 + : +- CometColumnarToRow + : +- 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 + +- CometColumnarToRow + +- 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 143 out of 332 eligible operators (43%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt index c67ddd5c51..ac3d312ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt @@ -1,178 +1,139 @@ -WholeStageCodegen (76) +WholeStageCodegen (22) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (21) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (6) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) + WholeStageCodegen (5) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] + CometExchange [s_store_sk] #3 + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometExchange [s_store_sk] #8 + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 + WholeStageCodegen (3) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + BroadcastExchange #9 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [cs_call_center_sk] #10 + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) + ReusedExchange [d_date_sk] #5 + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #11 + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] + CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] + CometExchange [wp_web_page_sk] #12 + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk] + CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [wp_web_page_sk] #13 + CometFilter [wp_web_page_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] + CometExchange [wp_web_page_sk] #15 + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk] + CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + ReusedExchange [wp_web_page_sk] #13 + WholeStageCodegen (13) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) + CometColumnarExchange [channel] #16 + WholeStageCodegen (12) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + WholeStageCodegen (20) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) + CometColumnarExchange #17 + WholeStageCodegen (19) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt index 7578c94177..82a5357f89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt @@ -1,76 +1,71 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- RowToColumnar (70) - +- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- * CometColumnarToRow (19) - : : +- CometColumnarExchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (35) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- * CometColumnarToRow (64) - +- CometColumnarExchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * CometColumnarToRow (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometFilter (47) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (59) +TakeOrderedAndProject (67) ++- * Project (66) + +- * CometColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometExchange (46) + : : +- CometFilter (45) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -129,308 +124,282 @@ Condition : isnull(sr_ticket_number#10) Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) CometColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) ReusedExchange [Reuses operator id: 76] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(16) Project [codegen id : 2] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(18) CometColumnarExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometColumnarToRow [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(19) CometExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 3] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) (24) CometExchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] (25) CometSort -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (27) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) (28) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] (29) CometExchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] (30) CometSort -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] (31) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter (32) CometFilter -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) (33) CometProject -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(34) CometColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(35) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#38, d_year#39] - -(36) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 5] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(38) HashAggregate [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(39) CometColumnarExchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(41) HashAggregate [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 7] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 7] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(48) CometExchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(46) CometExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(49) CometSort -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(52) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(53) CometExchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(54) CometSort -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) +(51) CometExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(57) CometProject -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] -(58) CometColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter -(59) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#65, d_year#66] +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) -(60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -(61) Project [codegen id : 9] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(62) HashAggregate [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight -(63) CometColumnarExchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -(64) CometColumnarToRow [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] -(65) HashAggregate [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] +(60) CometExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(66) Filter [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] -(67) Sort [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) -(68) SortMergeJoin [codegen id : 11] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] -(69) Project [codegen id : 11] -Output [13]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)))), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(70) RowToColumnar -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +(65) CometColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -(71) CometTakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ss_sold_year#23 ASC NULLS FIRST,ss_item_sk#1 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST,ss_qty#24 DESC NULLS LAST,ss_wc#25 DESC NULLS LAST,ss_sp#26 DESC NULLS LAST,other_chan_qty#85 ASC NULLS FIRST,other_chan_wholesale_cost#86 ASC NULLS FIRST,other_chan_sales_price#87 ASC NULLS FIRST,ratio#81 ASC NULLS FIRST], output=[ratio#81,store_qty#82,store_wholesale_cost#83,store_sales_price#84,other_chan_qty#85,other_chan_wholesale_cost#86,other_chan_sales_price#87]), [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87], 100, 0, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(66) Project [codegen id : 1] +Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -(72) CometColumnarToRow [codegen id : 12] -Input [7]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(67) TakeOrderedAndProject +Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(76) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index f19cb299e9..97a1bdbbaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -1,90 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Sort - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Sort - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project [COMET: Comet does not support Spark's BigDecimal rounding] + +- CometColumnarToRow + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 48 out of 76 eligible operators (63%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt index 8f1fd0f9c3..273db28e77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt @@ -1,104 +1,78 @@ -WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ratio,store_qty,store_wholesale_cost,store_sales_price,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - RowToColumnar - WholeStageCodegen (11) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] - InputAdapter - WholeStageCodegen (7) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (6) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] +TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] + WholeStageCodegen (1) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (10) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt index 7578c94177..82a5357f89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt @@ -1,76 +1,71 @@ == Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- RowToColumnar (70) - +- * Project (69) - +- * SortMergeJoin Inner (68) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (21) - : : +- * HashAggregate (20) - : : +- * CometColumnarToRow (19) - : : +- CometColumnarExchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (14) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (35) - +- * Sort (67) - +- * Filter (66) - +- * HashAggregate (65) - +- * CometColumnarToRow (64) - +- CometColumnarExchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * CometColumnarToRow (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometFilter (47) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (59) +TakeOrderedAndProject (67) ++- * Project (66) + +- * CometColumnarToRow (65) + +- CometSortMergeJoin (64) + :- CometProject (43) + : +- CometSortMergeJoin (42) + : :- CometSort (21) + : : +- CometHashAggregate (20) + : : +- CometExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) + : +- CometSort (41) + : +- CometFilter (40) + : +- CometHashAggregate (39) + : +- CometExchange (38) + : +- CometHashAggregate (37) + : +- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometProject (33) + : : +- CometFilter (32) + : : +- CometSortMergeJoin (31) + : : :- CometSort (25) + : : : +- CometExchange (24) + : : : +- CometFilter (23) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) + : : +- CometSort (30) + : : +- CometExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) + : +- ReusedExchange (34) + +- CometSort (63) + +- CometFilter (62) + +- CometHashAggregate (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometFilter (54) + : +- CometSortMergeJoin (53) + : :- CometSort (47) + : : +- CometExchange (46) + : : +- CometFilter (45) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) + : +- CometSort (52) + : +- CometExchange (51) + : +- CometProject (50) + : +- CometFilter (49) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) + +- ReusedExchange (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -129,308 +124,282 @@ Condition : isnull(sr_ticket_number#10) Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -(13) CometColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) ReusedExchange [Reuses operator id: 76] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(15) CometBroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: [d_date_sk#12, d_year#13] -(15) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#12, d_year#13] +Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight -(16) Project [codegen id : 2] -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +(17) CometProject Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -(17) HashAggregate [codegen id : 2] +(18) CometHashAggregate Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum#14, sum#15, sum#16] -Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] - -(18) CometColumnarExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometColumnarToRow [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(19) CometExchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(20) HashAggregate [codegen id : 3] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +(20) CometHashAggregate +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] -Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] -(21) Sort [codegen id : 3] -Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 +(21) CometSort +Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (23) CometFilter -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) (24) CometExchange -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] (25) CometSort -Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST] +Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (27) CometFilter -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) (28) CometProject -Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] +Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] (29) CometExchange -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] (30) CometSort -Input [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] +Input [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] (31) CometSortMergeJoin -Left output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] -Right output [2]: [wr_item_sk#35, wr_order_number#36] -Arguments: [ws_order_number#29, ws_item_sk#27], [wr_order_number#36, wr_item_sk#35], LeftOuter +Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [wr_item_sk#29, wr_order_number#30] +Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter (32) CometFilter -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Condition : isnull(wr_order_number#36) +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Condition : isnull(wr_order_number#30) (33) CometProject -Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] -Arguments: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33], [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(34) CometColumnarToRow [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] - -(35) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#38, d_year#39] - -(36) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 5] -Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] - -(38) HashAggregate [codegen id : 5] -Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum#40, sum#41, sum#42] -Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(39) CometColumnarExchange -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] - -(41) HashAggregate [codegen id : 6] -Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] -Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] -Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] -Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] -Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] - -(42) Filter [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Condition : (coalesce(ws_qty#51, 0) > 0) - -(43) Sort [codegen id : 6] -Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] -Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 7] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 7] -Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] -Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#32, d_year#33] + +(35) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [d_date_sk#32, d_year#33] +Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight + +(36) CometProject +Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] +Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] + +(37) CometHashAggregate +Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] + +(38) CometExchange +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(39) CometHashAggregate +Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] +Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] +Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] + +(40) CometFilter +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Condition : (coalesce(ws_qty#39, 0) > 0) + +(41) CometSort +Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] + +(42) CometSortMergeJoin +Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner + +(43) CometProject +Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] + +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) +(45) CometFilter +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) -(48) CometExchange -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(46) CometExchange +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(49) CometSort -Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST] +(47) CometSort +Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) - -(52) CometProject -Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] - -(53) CometExchange -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(54) CometSort -Input [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] +(49) CometFilter +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] -Right output [2]: [cr_item_sk#62, cr_order_number#63] -Arguments: [cs_order_number#56, cs_item_sk#55], [cr_order_number#63, cr_item_sk#62], LeftOuter +(50) CometProject +Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Condition : isnull(cr_order_number#63) +(51) CometExchange +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(57) CometProject -Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] -Arguments: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60], [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(52) CometSort +Input [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] -(58) CometColumnarToRow [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +(53) CometSortMergeJoin +Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [cr_item_sk#50, cr_order_number#51] +Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter -(59) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#65, d_year#66] +(54) CometFilter +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Condition : isnull(cr_order_number#51) -(60) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None +(55) CometProject +Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -(61) Project [codegen id : 9] -Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(62) HashAggregate [codegen id : 9] -Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum#67, sum#68, sum#69] -Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +(57) CometBroadcastHashJoin +Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight -(63) CometColumnarExchange -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(58) CometProject +Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] +Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -(64) CometColumnarToRow [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +(59) CometHashAggregate +Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] -(65) HashAggregate [codegen id : 10] -Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] -Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] -Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] -Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] -Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] +(60) CometExchange +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(66) Filter [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Condition : (coalesce(cs_qty#78, 0) > 0) +(61) CometHashAggregate +Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] +Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] +Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] -(67) Sort [codegen id : 10] -Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] -Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 +(62) CometFilter +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Condition : (coalesce(cs_qty#60, 0) > 0) -(68) SortMergeJoin [codegen id : 11] -Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] -Join type: Inner -Join condition: None +(63) CometSort +Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] -(69) Project [codegen id : 11] -Output [13]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)))), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +(64) CometSortMergeJoin +Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] +Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] +Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner -(70) RowToColumnar -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +(65) CometColumnarToRow [codegen id : 1] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -(71) CometTakeOrderedAndProject -Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ss_sold_year#23 ASC NULLS FIRST,ss_item_sk#1 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST,ss_qty#24 DESC NULLS LAST,ss_wc#25 DESC NULLS LAST,ss_sp#26 DESC NULLS LAST,other_chan_qty#85 ASC NULLS FIRST,other_chan_wholesale_cost#86 ASC NULLS FIRST,other_chan_sales_price#87 ASC NULLS FIRST,ratio#81 ASC NULLS FIRST], output=[ratio#81,store_qty#82,store_wholesale_cost#83,store_sales_price#84,other_chan_qty#85,other_chan_wholesale_cost#86,other_chan_sales_price#87]), [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87], 100, 0, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(66) Project [codegen id : 1] +Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -(72) CometColumnarToRow [codegen id : 12] -Input [7]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] +(67) TakeOrderedAndProject +Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] +Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(74) CometFilter +(69) CometFilter Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(76) BroadcastExchange +(71) BroadcastExchange Input [2]: [d_date_sk#12, d_year#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt index f19cb299e9..97a1bdbbaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt @@ -1,90 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- 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 - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Sort - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Sort - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + TakeOrderedAndProject [COMET: ] ++- Project [COMET: Comet does not support Spark's BigDecimal rounding] + +- CometColumnarToRow + +- 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 + : : : : +- CometColumnarToRow + : : : : +- 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 48 out of 76 eligible operators (63%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt index 8f1fd0f9c3..273db28e77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt @@ -1,104 +1,78 @@ -WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ratio,store_qty,store_wholesale_cost,store_sales_price,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - RowToColumnar - WholeStageCodegen (11) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] - InputAdapter - WholeStageCodegen (7) - Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_sold_year,ss_item_sk,ss_customer_sk] - HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (6) - Sort [ws_sold_year,ws_item_sk,ws_customer_sk] - Filter [ws_qty] - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 - WholeStageCodegen (5) - HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] - Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] +TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] + WholeStageCodegen (1) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + CometColumnarToRow + InputAdapter + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometExchange [ss_ticket_number,ss_item_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #6 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #7 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - WholeStageCodegen (10) - Sort [cs_sold_year,cs_item_sk,cs_customer_sk] - Filter [cs_qty] - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 - WholeStageCodegen (9) - HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #9 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #10 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #4 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #5 + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] + CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] + CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometExchange [ws_order_number,ws_item_sk] #7 + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number] + CometExchange [wr_order_number,wr_item_sk] #8 + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] + CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometExchange [cs_order_number,cs_item_sk] #10 + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_order_number,cr_item_sk] #11 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt index e22170be2f..f9901f13d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt @@ -1,133 +1,120 @@ == Physical Plan == -* CometColumnarToRow (129) -+- CometTakeOrderedAndProject (128) - +- CometHashAggregate (127) - +- CometColumnarExchange (126) - +- * HashAggregate (125) - +- Union (124) - :- * HashAggregate (109) - : +- * CometColumnarToRow (108) - : +- CometColumnarExchange (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometColumnarExchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (20) - : : : : +- * CometColumnarToRow (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (27) - : : : +- * CometColumnarToRow (26) - : : : +- CometProject (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) - : : +- BroadcastExchange (34) - : : +- * CometColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (72) - : : +- * CometColumnarToRow (71) - : : +- CometColumnarExchange (70) - : : +- * HashAggregate (69) - : : +- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : :- * Project (62) - : : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * CometColumnarToRow (52) - : : : : : : +- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometExchange (43) - : : : : : : : +- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- CometSort (49) - : : : : : : +- CometExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (60) - : : : : +- * CometColumnarToRow (59) - : : : : +- CometProject (58) - : : : : +- CometFilter (57) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (63) - : : +- ReusedExchange (66) - : +- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (87) - : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : :- * CometColumnarToRow (84) - : : : : : +- CometProject (83) - : : : : : +- CometSortMergeJoin (82) - : : : : : :- CometSort (76) - : : : : : : +- CometExchange (75) - : : : : : : +- CometFilter (74) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (73) - : : : : : +- CometSort (81) - : : : : : +- CometExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (77) - : : : : +- ReusedExchange (85) - : : : +- BroadcastExchange (92) - : : : +- * CometColumnarToRow (91) - : : : +- CometProject (90) - : : : +- CometFilter (89) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (88) - : : +- ReusedExchange (95) - : +- ReusedExchange (98) - :- * HashAggregate (116) - : +- * CometColumnarToRow (115) - : +- CometColumnarExchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- ReusedExchange (110) - +- * HashAggregate (123) - +- * CometColumnarToRow (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- ReusedExchange (117) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -182,583 +169,515 @@ Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#1 Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(12) CometColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct + +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(13) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#14] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight -(15) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(18) CometProject -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: [s_store_sk#15, s_store_id#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#17] +(20) CometProject +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] -(19) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#15, s_store_id#17] +(21) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#18] +Arguments: [s_store_sk#16, s_store_id#18] -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#18] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(23) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -(22) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#17] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_current_price#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#19, i_current_price#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [i_item_sk#18, i_current_price#19] -Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) - -(25) CometProject -Input [2]: [i_item_sk#18, i_current_price#19] -Arguments: [i_item_sk#18], [i_item_sk#18] +(25) CometFilter +Input [2]: [i_item_sk#19, i_current_price#20] +Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) -(26) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#18] +(26) CometProject +Input [2]: [i_item_sk#19, i_current_price#20] +Arguments: [i_item_sk#19], [i_item_sk#19] -(27) BroadcastExchange -Input [1]: [i_item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(27) CometBroadcastExchange +Input [1]: [i_item_sk#19] +Arguments: [i_item_sk#19] -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Right output [1]: [i_item_sk#19] +Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight -(29) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +(29) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] (30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +Output [2]: [p_promo_sk#21, p_channel_tv#22] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (31) CometFilter -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#21, 1, true, false, true) = N) AND isnotnull(p_promo_sk#20)) +Input [2]: [p_promo_sk#21, p_channel_tv#22] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) (32) CometProject -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Arguments: [p_promo_sk#20], [p_promo_sk#20] +Input [2]: [p_promo_sk#21, p_channel_tv#22] +Arguments: [p_promo_sk#21], [p_promo_sk#21] -(33) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#20] +(33) CometBroadcastExchange +Input [1]: [p_promo_sk#21] +Arguments: [p_promo_sk#21] -(34) BroadcastExchange -Input [1]: [p_promo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Right output [1]: [p_promo_sk#21] +Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -(36) Project [codegen id : 5] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] - -(37) HashAggregate [codegen id : 5] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Keys [1]: [s_store_id#17] +(36) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Keys [1]: [s_store_id#18] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] -Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] - -(38) CometColumnarExchange -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometColumnarToRow [codegen id : 6] -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +(37) CometExchange +Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] +Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) HashAggregate [codegen id : 6] -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -Keys [1]: [s_store_id#17] +(38) CometHashAggregate +Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] +Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#17) AS id#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#37, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#38, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#39] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(42) CometFilter -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(43) CometExchange -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(44) CometSort -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) - -(47) CometProject -Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(48) CometExchange -Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(49) CometSort -Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(51) CometProject -Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(52) CometColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(53) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#53] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(55) Project [codegen id : 11] -Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] -Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -Condition : isnotnull(cp_catalog_page_sk#54) +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) + +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] + +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] + +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight (58) CometProject -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#56], [cp_catalog_page_sk#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#55, 16, true, false, true) AS cp_catalog_page_id#56] - -(59) CometColumnarToRow [codegen id : 8] -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] - -(60) BroadcastExchange -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_catalog_page_sk#40] -Right keys [1]: [cp_catalog_page_sk#54] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 11] -Output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#56] - -(63) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#57] - -(64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#41] -Right keys [1]: [i_item_sk#57] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 11] -Output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, i_item_sk#57] - -(66) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#58] - -(67) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_promo_sk#42] -Right keys [1]: [p_promo_sk#58] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 11] -Output [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, p_promo_sk#58] - -(69) HashAggregate [codegen id : 11] -Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Keys [1]: [cp_catalog_page_id#56] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#59, sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] - -(70) CometColumnarExchange -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cp_catalog_page_id#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(71) CometColumnarToRow [codegen id : 12] -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] - -(72) HashAggregate [codegen id : 12] -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Keys [1]: [cp_catalog_page_id#56] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#69, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71] -Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#56) AS id#73, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#69,17,2) AS sales#74, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70 AS returns#75, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71 AS profit#76] - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] + +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight + +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] + +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight + +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] + +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] + +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(74) CometFilter -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Condition : ((isnotnull(ws_web_site_sk#78) AND isnotnull(ws_item_sk#77)) AND isnotnull(ws_promo_sk#79)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(75) CometExchange -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Arguments: hashpartitioning(ws_item_sk#77, ws_order_number#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometSort -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83], [ws_item_sk#77 ASC NULLS FIRST, ws_order_number#80 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(78) CometFilter -Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] -Condition : (isnotnull(wr_item_sk#85) AND isnotnull(wr_order_number#86)) - -(79) CometProject -Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] -Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(80) CometExchange -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: hashpartitioning(wr_item_sk#85, wr_order_number#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(81) CometSort -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85 ASC NULLS FIRST, wr_order_number#86 ASC NULLS FIRST] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(82) CometSortMergeJoin -Left output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Right output [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [ws_item_sk#77, ws_order_number#80], [wr_item_sk#85, wr_order_number#86], LeftOuter +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(83) CometProject -Input [11]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88], [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(84) CometColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(85) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#90] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(86) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(87) Project [codegen id : 17] -Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88] -Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88, d_date_sk#90] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#91, web_site_id#92] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(89) CometFilter -Input [2]: [web_site_sk#91, web_site_id#92] -Condition : isnotnull(web_site_sk#91) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) + +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] + +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] + +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight + +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] + +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight (90) CometProject -Input [2]: [web_site_sk#91, web_site_id#92] -Arguments: [web_site_sk#91, web_site_id#93], [web_site_sk#91, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#92, 16, true, false, true) AS web_site_id#93] - -(91) CometColumnarToRow [codegen id : 14] -Input [2]: [web_site_sk#91, web_site_id#93] - -(92) BroadcastExchange -Input [2]: [web_site_sk#91, web_site_id#93] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -(93) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_site_sk#78] -Right keys [1]: [web_site_sk#91] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 17] -Output [7]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_sk#91, web_site_id#93] - -(95) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#94] - -(96) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [i_item_sk#94] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 17] -Output [6]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [8]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, i_item_sk#94] - -(98) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#95] - -(99) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_promo_sk#79] -Right keys [1]: [p_promo_sk#95] -Join type: Inner -Join condition: None - -(100) Project [codegen id : 17] -Output [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [7]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, p_promo_sk#95] - -(101) HashAggregate [codegen id : 17] -Input [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Keys [1]: [web_site_id#93] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#81)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(102) CometColumnarExchange -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(web_site_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) CometColumnarToRow [codegen id : 18] -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(104) HashAggregate [codegen id : 18] -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [web_site_id#93] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#81)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#81))#106, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108] -Results [5]: [web channel AS channel#109, concat(web_site, web_site_id#93) AS id#110, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#106,17,2) AS sales#111, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107 AS returns#112, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108 AS profit#113] - -(105) Union - -(106) HashAggregate [codegen id : 19] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(107) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(108) CometColumnarToRow [codegen id : 20] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(109) HashAggregate [codegen id : 20] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#126 as decimal(37,2)) AS sales#129, cast(sum(returns#38)#127 as decimal(38,2)) AS returns#130, cast(sum(profit#39)#128 as decimal(38,2)) AS profit#131] - -(110) ReusedExchange [Reuses operator id: 107] -Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(111) CometColumnarToRow [codegen id : 40] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(112) HashAggregate [codegen id : 40] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [4]: [channel#35, sum(sales#37)#126 AS sales#132, sum(returns#38)#127 AS returns#133, sum(profit#39)#128 AS profit#134] - -(113) HashAggregate [codegen id : 40] -Input [4]: [channel#35, sales#132, returns#133, profit#134] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(114) CometColumnarExchange -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) CometColumnarToRow [codegen id : 41] -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(116) HashAggregate [codegen id : 41] -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#147, sum(returns#133)#148, sum(profit#134)#149] -Results [5]: [channel#35, null AS id#150, sum(sales#132)#147 AS sales#151, sum(returns#133)#148 AS returns#152, sum(profit#134)#149 AS profit#153] - -(117) ReusedExchange [Reuses operator id: 107] -Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(118) CometColumnarToRow [codegen id : 61] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(119) HashAggregate [codegen id : 61] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [3]: [sum(sales#37)#126 AS sales#154, sum(returns#38)#127 AS returns#155, sum(profit#39)#128 AS profit#156] - -(120) HashAggregate [codegen id : 61] -Input [3]: [sales#154, returns#155, profit#156] +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] + +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight + +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(96) CometHashAggregate +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] -Aggregate Attributes [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Results [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] - -(121) CometColumnarExchange -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] -(122) CometColumnarToRow [codegen id : 62] -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +(109) CometExchange +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(123) HashAggregate [codegen id : 62] -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +(110) CometHashAggregate +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] Keys: [] -Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] -Aggregate Attributes [3]: [sum(sales#154)#169, sum(returns#155)#170, sum(profit#156)#171] -Results [5]: [null AS channel#172, null AS id#173, sum(sales#154)#169 AS sales#174, sum(returns#155)#170 AS returns#175, sum(profit#156)#171 AS profit#176] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] -(124) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] +Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] -(125) HashAggregate [codegen id : 63] -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -(126) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Arguments: hashpartitioning(channel#35, id#36, sales#129, returns#130, profit#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(127) CometHashAggregate -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -(128) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#129,returns#130,profit#131]), [channel#35, id#36, sales#129, returns#130, profit#131], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#129, returns#130, profit#131] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] -(129) CometColumnarToRow [codegen id : 64] -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometProject (132) - +- CometFilter (131) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (130) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(130) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#177] +(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter -Input [2]: [d_date_sk#14, d_date#177] -Condition : (((isnotnull(d_date#177) AND (d_date#177 >= 1998-08-04)) AND (d_date#177 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(118) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(132) CometProject -Input [2]: [d_date_sk#14, d_date#177] +(119) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(133) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(134) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index 7c1d834ff9..0d6844d93c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -1,451 +1,392 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow + +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 233 out of 386 eligible operators (60%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt index 18b1a637b1..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt @@ -1,189 +1,128 @@ -WholeStageCodegen (64) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (63) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (20) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #10 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #14 - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (14) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (41) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #18 - WholeStageCodegen (40) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (62) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #19 - WholeStageCodegen (61) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index e22170be2f..f9901f13d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -1,133 +1,120 @@ == Physical Plan == -* CometColumnarToRow (129) -+- CometTakeOrderedAndProject (128) - +- CometHashAggregate (127) - +- CometColumnarExchange (126) - +- * HashAggregate (125) - +- Union (124) - :- * HashAggregate (109) - : +- * CometColumnarToRow (108) - : +- CometColumnarExchange (107) - : +- * HashAggregate (106) - : +- Union (105) - : :- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometColumnarExchange (38) - : : +- * HashAggregate (37) - : : +- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (22) - : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- ReusedExchange (13) - : : : : +- BroadcastExchange (20) - : : : : +- * CometColumnarToRow (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - : : : +- BroadcastExchange (27) - : : : +- * CometColumnarToRow (26) - : : : +- CometProject (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (23) - : : +- BroadcastExchange (34) - : : +- * CometColumnarToRow (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (72) - : : +- * CometColumnarToRow (71) - : : +- CometColumnarExchange (70) - : : +- * HashAggregate (69) - : : +- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Project (65) - : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : :- * Project (62) - : : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : : :- * Project (55) - : : : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : : : :- * CometColumnarToRow (52) - : : : : : : +- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometExchange (43) - : : : : : : : +- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- CometSort (49) - : : : : : : +- CometExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (53) - : : : : +- BroadcastExchange (60) - : : : : +- * CometColumnarToRow (59) - : : : : +- CometProject (58) - : : : : +- CometFilter (57) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (56) - : : : +- ReusedExchange (63) - : : +- ReusedExchange (66) - : +- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- * Project (100) - : +- * BroadcastHashJoin Inner BuildRight (99) - : :- * Project (97) - : : +- * BroadcastHashJoin Inner BuildRight (96) - : : :- * Project (94) - : : : +- * BroadcastHashJoin Inner BuildRight (93) - : : : :- * Project (87) - : : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : : :- * CometColumnarToRow (84) - : : : : : +- CometProject (83) - : : : : : +- CometSortMergeJoin (82) - : : : : : :- CometSort (76) - : : : : : : +- CometExchange (75) - : : : : : : +- CometFilter (74) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (73) - : : : : : +- CometSort (81) - : : : : : +- CometExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (77) - : : : : +- ReusedExchange (85) - : : : +- BroadcastExchange (92) - : : : +- * CometColumnarToRow (91) - : : : +- CometProject (90) - : : : +- CometFilter (89) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (88) - : : +- ReusedExchange (95) - : +- ReusedExchange (98) - :- * HashAggregate (116) - : +- * CometColumnarToRow (115) - : +- CometColumnarExchange (114) - : +- * HashAggregate (113) - : +- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- ReusedExchange (110) - +- * HashAggregate (123) - +- * CometColumnarToRow (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- ReusedExchange (117) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -182,583 +169,515 @@ Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#1 Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -(12) CometColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct + +(13) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(13) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#14] +(14) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight -(15) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +(17) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_store_id#16] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_store_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [s_store_sk#15, s_store_id#16] -Condition : isnotnull(s_store_sk#15) +(19) CometFilter +Input [2]: [s_store_sk#16, s_store_id#17] +Condition : isnotnull(s_store_sk#16) -(18) CometProject -Input [2]: [s_store_sk#15, s_store_id#16] -Arguments: [s_store_sk#15, s_store_id#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#17] +(20) CometProject +Input [2]: [s_store_sk#16, s_store_id#17] +Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] -(19) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#15, s_store_id#17] +(21) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_store_id#18] +Arguments: [s_store_sk#16, s_store_id#18] -(20) BroadcastExchange -Input [2]: [s_store_sk#15, s_store_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Right output [2]: [s_store_sk#16, s_store_id#18] +Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(23) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -(22) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#17] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_current_price#19] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [2]: [i_item_sk#19, i_current_price#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [i_item_sk#18, i_current_price#19] -Condition : ((isnotnull(i_current_price#19) AND (i_current_price#19 > 50.00)) AND isnotnull(i_item_sk#18)) - -(25) CometProject -Input [2]: [i_item_sk#18, i_current_price#19] -Arguments: [i_item_sk#18], [i_item_sk#18] +(25) CometFilter +Input [2]: [i_item_sk#19, i_current_price#20] +Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) -(26) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#18] +(26) CometProject +Input [2]: [i_item_sk#19, i_current_price#20] +Arguments: [i_item_sk#19], [i_item_sk#19] -(27) BroadcastExchange -Input [1]: [i_item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(27) CometBroadcastExchange +Input [1]: [i_item_sk#19] +Arguments: [i_item_sk#19] -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Right output [1]: [i_item_sk#19] +Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight -(29) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, i_item_sk#18] +(29) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] +Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] (30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#20, p_channel_tv#21] +Output [2]: [p_promo_sk#21, p_channel_tv#22] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct (31) CometFilter -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#21, 1, true, false, true) = N) AND isnotnull(p_promo_sk#20)) +Input [2]: [p_promo_sk#21, p_channel_tv#22] +Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) (32) CometProject -Input [2]: [p_promo_sk#20, p_channel_tv#21] -Arguments: [p_promo_sk#20], [p_promo_sk#20] +Input [2]: [p_promo_sk#21, p_channel_tv#22] +Arguments: [p_promo_sk#21], [p_promo_sk#21] -(33) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#20] +(33) CometBroadcastExchange +Input [1]: [p_promo_sk#21] +Arguments: [p_promo_sk#21] -(34) BroadcastExchange -Input [1]: [p_promo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Right output [1]: [p_promo_sk#21] +Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] +Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -(36) Project [codegen id : 5] -Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17, p_promo_sk#20] - -(37) HashAggregate [codegen id : 5] -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#17] -Keys [1]: [s_store_id#17] +(36) CometHashAggregate +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] +Keys [1]: [s_store_id#18] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] -Results [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] - -(38) CometColumnarExchange -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(39) CometColumnarToRow [codegen id : 6] -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] +(37) CometExchange +Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] +Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) HashAggregate [codegen id : 6] -Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] -Keys [1]: [s_store_id#17] +(38) CometHashAggregate +Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] +Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#32, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#17) AS id#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS sales#37, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#33 AS returns#38, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#34 AS profit#39] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(42) CometFilter -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : ((isnotnull(cs_catalog_page_sk#40) AND isnotnull(cs_item_sk#41)) AND isnotnull(cs_promo_sk#42)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(43) CometExchange -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: hashpartitioning(cs_item_sk#41, cs_order_number#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(44) CometSort -Input [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46], [cs_item_sk#41 ASC NULLS FIRST, cs_order_number#43 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(46) CometFilter -Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_item_sk#48) AND isnotnull(cr_order_number#49)) - -(47) CometProject -Input [5]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(48) CometExchange -Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: hashpartitioning(cr_item_sk#48, cr_order_number#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(49) CometSort -Input [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51], [cr_item_sk#48 ASC NULLS FIRST, cr_order_number#49 ASC NULLS FIRST] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Right output [4]: [cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cs_item_sk#41, cs_order_number#43], [cr_item_sk#48, cr_order_number#49], LeftOuter +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(51) CometProject -Input [11]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_order_number#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_item_sk#48, cr_order_number#49, cr_return_amount#50, cr_net_loss#51] -Arguments: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51], [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(52) CometColumnarToRow [codegen id : 11] -Input [8]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(53) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#53] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(55) Project [codegen id : 11] -Output [7]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51] -Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46, cr_return_amount#50, cr_net_loss#51, d_date_sk#53] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -Condition : isnotnull(cp_catalog_page_sk#54) +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) + +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] + +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] + +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight (58) CometProject -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#56], [cp_catalog_page_sk#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#55, 16, true, false, true) AS cp_catalog_page_id#56] - -(59) CometColumnarToRow [codegen id : 8] -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] - -(60) BroadcastExchange -Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(61) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_catalog_page_sk#40] -Right keys [1]: [cp_catalog_page_sk#54] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 11] -Output [7]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [9]: [cs_catalog_page_sk#40, cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_sk#54, cp_catalog_page_id#56] - -(63) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#57] - -(64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#41] -Right keys [1]: [i_item_sk#57] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 11] -Output [6]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [8]: [cs_item_sk#41, cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, i_item_sk#57] - -(66) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#58] - -(67) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_promo_sk#42] -Right keys [1]: [p_promo_sk#58] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 11] -Output [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Input [7]: [cs_promo_sk#42, cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56, p_promo_sk#58] - -(69) HashAggregate [codegen id : 11] -Input [5]: [cs_ext_sales_price#44, cs_net_profit#45, cr_return_amount#50, cr_net_loss#51, cp_catalog_page_id#56] -Keys [1]: [cp_catalog_page_id#56] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#44)), partial_sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#59, sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] - -(70) CometColumnarExchange -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cp_catalog_page_id#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(71) CometColumnarToRow [codegen id : 12] -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] - -(72) HashAggregate [codegen id : 12] -Input [6]: [cp_catalog_page_id#56, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Keys [1]: [cp_catalog_page_id#56] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#44)), sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00)), sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#44))#69, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71] -Results [5]: [catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#56) AS id#73, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#44))#69,17,2) AS sales#74, sum(coalesce(cast(cr_return_amount#50 as decimal(12,2)), 0.00))#70 AS returns#75, sum((cs_net_profit#45 - coalesce(cast(cr_net_loss#51 as decimal(12,2)), 0.00)))#71 AS profit#76] - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] + +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight + +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] + +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight + +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] + +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] + +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] + +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(74) CometFilter -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Condition : ((isnotnull(ws_web_site_sk#78) AND isnotnull(ws_item_sk#77)) AND isnotnull(ws_promo_sk#79)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(75) CometExchange -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Arguments: hashpartitioning(ws_item_sk#77, ws_order_number#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometSort -Input [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83], [ws_item_sk#77 ASC NULLS FIRST, ws_order_number#80 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(78) CometFilter -Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] -Condition : (isnotnull(wr_item_sk#85) AND isnotnull(wr_order_number#86)) - -(79) CometProject -Input [5]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88, wr_returned_date_sk#89] -Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(80) CometExchange -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: hashpartitioning(wr_item_sk#85, wr_order_number#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(81) CometSort -Input [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88], [wr_item_sk#85 ASC NULLS FIRST, wr_order_number#86 ASC NULLS FIRST] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(82) CometSortMergeJoin -Left output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83] -Right output [4]: [wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [ws_item_sk#77, ws_order_number#80], [wr_item_sk#85, wr_order_number#86], LeftOuter +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(83) CometProject -Input [11]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_order_number#80, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_item_sk#85, wr_order_number#86, wr_return_amt#87, wr_net_loss#88] -Arguments: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88], [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(84) CometColumnarToRow [codegen id : 17] -Input [8]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(85) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#90] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(86) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] -Join type: Inner -Join condition: None +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(87) Project [codegen id : 17] -Output [7]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88] -Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, ws_sold_date_sk#83, wr_return_amt#87, wr_net_loss#88, d_date_sk#90] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#91, web_site_id#92] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(89) CometFilter -Input [2]: [web_site_sk#91, web_site_id#92] -Condition : isnotnull(web_site_sk#91) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) + +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] + +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] + +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight + +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] + +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight (90) CometProject -Input [2]: [web_site_sk#91, web_site_id#92] -Arguments: [web_site_sk#91, web_site_id#93], [web_site_sk#91, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#92, 16, true, false, true) AS web_site_id#93] - -(91) CometColumnarToRow [codegen id : 14] -Input [2]: [web_site_sk#91, web_site_id#93] - -(92) BroadcastExchange -Input [2]: [web_site_sk#91, web_site_id#93] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -(93) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_site_sk#78] -Right keys [1]: [web_site_sk#91] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 17] -Output [7]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [9]: [ws_item_sk#77, ws_web_site_sk#78, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_sk#91, web_site_id#93] - -(95) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#94] - -(96) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [i_item_sk#94] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 17] -Output [6]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [8]: [ws_item_sk#77, ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, i_item_sk#94] - -(98) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#95] - -(99) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_promo_sk#79] -Right keys [1]: [p_promo_sk#95] -Join type: Inner -Join condition: None - -(100) Project [codegen id : 17] -Output [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Input [7]: [ws_promo_sk#79, ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93, p_promo_sk#95] - -(101) HashAggregate [codegen id : 17] -Input [5]: [ws_ext_sales_price#81, ws_net_profit#82, wr_return_amt#87, wr_net_loss#88, web_site_id#93] -Keys [1]: [web_site_id#93] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#81)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] -Aggregate Attributes [5]: [sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(102) CometColumnarExchange -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(web_site_id#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) CometColumnarToRow [codegen id : 18] -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(104) HashAggregate [codegen id : 18] -Input [6]: [web_site_id#93, sum#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [web_site_id#93] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#81)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#81))#106, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108] -Results [5]: [web channel AS channel#109, concat(web_site, web_site_id#93) AS id#110, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#81))#106,17,2) AS sales#111, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#107 AS returns#112, sum((ws_net_profit#82 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#108 AS profit#113] - -(105) Union - -(106) HashAggregate [codegen id : 19] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Results [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(107) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(108) CometColumnarToRow [codegen id : 20] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(109) HashAggregate [codegen id : 20] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#126 as decimal(37,2)) AS sales#129, cast(sum(returns#38)#127 as decimal(38,2)) AS returns#130, cast(sum(profit#39)#128 as decimal(38,2)) AS profit#131] - -(110) ReusedExchange [Reuses operator id: 107] -Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(111) CometColumnarToRow [codegen id : 40] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(112) HashAggregate [codegen id : 40] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [4]: [channel#35, sum(sales#37)#126 AS sales#132, sum(returns#38)#127 AS returns#133, sum(profit#39)#128 AS profit#134] - -(113) HashAggregate [codegen id : 40] -Input [4]: [channel#35, sales#132, returns#133, profit#134] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(114) CometColumnarExchange -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) CometColumnarToRow [codegen id : 41] -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(116) HashAggregate [codegen id : 41] -Input [7]: [channel#35, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#147, sum(returns#133)#148, sum(profit#134)#149] -Results [5]: [channel#35, null AS id#150, sum(sales#132)#147 AS sales#151, sum(returns#133)#148 AS returns#152, sum(profit#134)#149 AS profit#153] - -(117) ReusedExchange [Reuses operator id: 107] -Output [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(118) CometColumnarToRow [codegen id : 61] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(119) HashAggregate [codegen id : 61] -Input [8]: [channel#35, id#36, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#126, sum(returns#38)#127, sum(profit#39)#128] -Results [3]: [sum(sales#37)#126 AS sales#154, sum(returns#38)#127 AS returns#155, sum(profit#39)#128 AS profit#156] - -(120) HashAggregate [codegen id : 61] -Input [3]: [sales#154, returns#155, profit#156] +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] + +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight + +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(96) CometHashAggregate +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#154), partial_sum(returns#155), partial_sum(profit#156)] -Aggregate Attributes [6]: [sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Results [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] - -(121) CometColumnarExchange -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] -(122) CometColumnarToRow [codegen id : 62] -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +(109) CometExchange +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(123) HashAggregate [codegen id : 62] -Input [6]: [sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168] +(110) CometHashAggregate +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] Keys: [] -Functions [3]: [sum(sales#154), sum(returns#155), sum(profit#156)] -Aggregate Attributes [3]: [sum(sales#154)#169, sum(returns#155)#170, sum(profit#156)#171] -Results [5]: [null AS channel#172, null AS id#173, sum(sales#154)#169 AS sales#174, sum(returns#155)#170 AS returns#175, sum(profit#156)#171 AS profit#176] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] -(124) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] +Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] -(125) HashAggregate [codegen id : 63] -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -(126) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Arguments: hashpartitioning(channel#35, id#36, sales#129, returns#130, profit#131, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(127) CometHashAggregate -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Keys [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -(128) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#129,returns#130,profit#131]), [channel#35, id#36, sales#129, returns#130, profit#131], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#129, returns#130, profit#131] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] -(129) CometColumnarToRow [codegen id : 64] -Input [5]: [channel#35, id#36, sales#129, returns#130, profit#131] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometProject (132) - +- CometFilter (131) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (130) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(130) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#177] +(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter -Input [2]: [d_date_sk#14, d_date#177] -Condition : (((isnotnull(d_date#177) AND (d_date#177 >= 1998-08-04)) AND (d_date#177 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) +(118) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(132) CometProject -Input [2]: [d_date_sk#14, d_date#177] +(119) CometProject +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(133) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(134) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt index 7c1d834ff9..0d6844d93c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt @@ -1,451 +1,392 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow + +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 + : : : : : : : +- CometColumnarToRow + : : : : : : : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- 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 - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- 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 - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- 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 - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- 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 + : : : : : : +- CometColumnarToRow + : : : : : : +- 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 233 out of 386 eligible operators (60%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index 18b1a637b1..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -1,189 +1,128 @@ -WholeStageCodegen (64) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (63) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (20) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #10 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] - Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (18) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #14 - WholeStageCodegen (17) - HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] - Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_promo_sk,p_promo_sk] - Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (14) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - InputAdapter - ReusedExchange [i_item_sk] #8 - InputAdapter - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (41) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #18 - WholeStageCodegen (40) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (62) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #19 - WholeStageCodegen (61) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt index 586f9cd0f7..b623de85d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt @@ -1,47 +1,41 @@ == Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- RowToColumnar (41) - +- * Project (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- Union (32) - :- * HashAggregate (17) - : +- * CometColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- ReusedExchange (25) +TakeOrderedAndProject (37) ++- * Project (36) + +- Window (35) + +- * CometColumnarToRow (34) + +- CometSort (33) + +- CometExchange (32) + +- CometHashAggregate (31) + +- CometExchange (30) + +- CometHashAggregate (29) + +- CometUnion (28) + :- CometHashAggregate (17) + : +- CometExchange (16) + : +- CometHashAggregate (15) + : +- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometHashAggregate (19) + : +- ReusedExchange (18) + +- CometHashAggregate (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometHashAggregate (24) + +- ReusedExchange (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -56,220 +50,192 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] +(11) CometProject +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#10, i_category#11] +Arguments: [i_item_sk#7, i_class#10, i_category#11] -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#10, i_category#11] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#9, i_category#10] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] +(14) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] +Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] -(14) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] +(15) CometHashAggregate +Input [3]: [ws_net_paid#2, i_class#10, i_category#11] +Keys [2]: [i_category#11, i_class#10] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_category#10, i_class#9, sum#12] -(15) CometColumnarExchange -Input [3]: [i_category#10, i_class#9, sum#12] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(16) CometExchange +Input [3]: [i_category#11, i_class#10, sum#12] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] - -(17) HashAggregate [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] -Keys [2]: [i_category#10, i_class#9] +(17) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#12] +Keys [2]: [i_category#11, i_class#10] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] - -(18) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#20] - -(19) CometColumnarToRow [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] - -(20) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] - -(21) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#27] -Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] - -(25) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#33] - -(26) CometColumnarToRow [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] - -(27) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] - -(28) HashAggregate [codegen id : 13] -Input [1]: [total_sum#35] -Keys: [] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] -(29) CometColumnarExchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#11, i_class#10, sum#13] + +(19) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#13] +Keys [2]: [i_category#11, i_class#10] +Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] + +(20) CometHashAggregate +Input [2]: [total_sum#15, i_category#11] +Keys [1]: [i_category#11] +Functions [1]: [partial_sum(total_sum#15)] + +(21) CometExchange +Input [3]: [i_category#11, sum#16, isEmpty#17] +Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(30) CometColumnarToRow [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +(22) CometHashAggregate +Input [3]: [i_category#11, sum#16, isEmpty#17] +Keys [1]: [i_category#11] +Functions [1]: [sum(total_sum#15)] -(31) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#11, i_class#10, sum#18] + +(24) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#18] +Keys [2]: [i_category#11, i_class#10] +Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] + +(25) CometHashAggregate +Input [1]: [total_sum#20] Keys: [] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] +Functions [1]: [partial_sum(total_sum#20)] -(32) Union +(26) CometExchange +Input [2]: [sum#21, isEmpty#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 15] -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +(27) CometHashAggregate +Input [2]: [sum#21, isEmpty#22] +Keys: [] +Functions [1]: [sum(total_sum#20)] -(34) CometColumnarExchange -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(28) CometUnion +Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] +Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] -(35) CometHashAggregate -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +(29) CometHashAggregate +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] Functions: [] -(36) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(30) CometExchange +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(37) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] +(31) CometHashAggregate +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Functions: [] -(38) CometColumnarToRow [codegen id : 16] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +(32) CometExchange +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(39) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] +(33) CometSort +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] -(40) Project [codegen id : 17] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] +(34) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -(41) RowToColumnar -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(35) Window +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] -(42) CometTakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#48 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#48]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(36) Project [codegen id : 2] +Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] -(43) CometColumnarToRow [codegen id : 18] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(37) TakeOrderedAndProject +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#49] +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#49] -Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) +(39) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [2]: [d_date_sk#5, d_month_seq#49] +(40) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 156bf16204..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -1,103 +1,87 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 81 eligible operators (53%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt index c53fb1296c..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt @@ -1,75 +1,49 @@ -WholeStageCodegen (18) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (17) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt index 586f9cd0f7..b623de85d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt @@ -1,47 +1,41 @@ == Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- RowToColumnar (41) - +- * Project (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- Union (32) - :- * HashAggregate (17) - : +- * CometColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- ReusedExchange (25) +TakeOrderedAndProject (37) ++- * Project (36) + +- Window (35) + +- * CometColumnarToRow (34) + +- CometSort (33) + +- CometExchange (32) + +- CometHashAggregate (31) + +- CometExchange (30) + +- CometHashAggregate (29) + +- CometUnion (28) + :- CometHashAggregate (17) + : +- CometExchange (16) + : +- CometHashAggregate (15) + : +- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometHashAggregate (19) + : +- ReusedExchange (18) + +- CometHashAggregate (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometHashAggregate (24) + +- ReusedExchange (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -56,220 +50,192 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] +(11) CometProject +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#10, i_category#11] +Arguments: [i_item_sk#7, i_class#10, i_category#11] -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#10, i_category#11] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#9, i_category#10] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] +(14) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] +Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] -(14) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] +(15) CometHashAggregate +Input [3]: [ws_net_paid#2, i_class#10, i_category#11] +Keys [2]: [i_category#11, i_class#10] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_category#10, i_class#9, sum#12] -(15) CometColumnarExchange -Input [3]: [i_category#10, i_class#9, sum#12] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(16) CometExchange +Input [3]: [i_category#11, i_class#10, sum#12] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] - -(17) HashAggregate [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] -Keys [2]: [i_category#10, i_class#9] +(17) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#12] +Keys [2]: [i_category#11, i_class#10] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] - -(18) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#20] - -(19) CometColumnarToRow [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] - -(20) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] - -(21) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#27] -Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] - -(25) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#33] - -(26) CometColumnarToRow [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] - -(27) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] - -(28) HashAggregate [codegen id : 13] -Input [1]: [total_sum#35] -Keys: [] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] -(29) CometColumnarExchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#11, i_class#10, sum#13] + +(19) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#13] +Keys [2]: [i_category#11, i_class#10] +Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] + +(20) CometHashAggregate +Input [2]: [total_sum#15, i_category#11] +Keys [1]: [i_category#11] +Functions [1]: [partial_sum(total_sum#15)] + +(21) CometExchange +Input [3]: [i_category#11, sum#16, isEmpty#17] +Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(30) CometColumnarToRow [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +(22) CometHashAggregate +Input [3]: [i_category#11, sum#16, isEmpty#17] +Keys [1]: [i_category#11] +Functions [1]: [sum(total_sum#15)] -(31) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#11, i_class#10, sum#18] + +(24) CometHashAggregate +Input [3]: [i_category#11, i_class#10, sum#18] +Keys [2]: [i_category#11, i_class#10] +Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] + +(25) CometHashAggregate +Input [1]: [total_sum#20] Keys: [] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] +Functions [1]: [partial_sum(total_sum#20)] -(32) Union +(26) CometExchange +Input [2]: [sum#21, isEmpty#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) HashAggregate [codegen id : 15] -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +(27) CometHashAggregate +Input [2]: [sum#21, isEmpty#22] +Keys: [] +Functions [1]: [sum(total_sum#20)] -(34) CometColumnarExchange -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(28) CometUnion +Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] +Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] -(35) CometHashAggregate -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] +(29) CometHashAggregate +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] Functions: [] -(36) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(30) CometExchange +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(37) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] +(31) CometHashAggregate +Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] +Functions: [] -(38) CometColumnarToRow [codegen id : 16] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] +(32) CometExchange +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(39) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] +(33) CometSort +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] -(40) Project [codegen id : 17] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] +(34) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -(41) RowToColumnar -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(35) Window +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] +Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] -(42) CometTakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#48 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#48]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(36) Project [codegen id : 2] +Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] -(43) CometColumnarToRow [codegen id : 18] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] +(37) TakeOrderedAndProject +Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#49] +(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#49] -Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) +(39) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [2]: [d_date_sk#5, d_month_seq#49] +(40) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt index 156bf16204..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt @@ -1,103 +1,87 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- RowToColumnar - +- Project - +- Window [COMET: Window expressions are not supported] - +- CometColumnarToRow - +- CometSort + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : : : +- CometColumnarToRow + : : : +- 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 + : : +- CometColumnarToRow + : : +- 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 43 out of 81 eligible operators (53%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt index c53fb1296c..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt @@ -1,75 +1,49 @@ -WholeStageCodegen (18) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - RowToColumnar - WholeStageCodegen (17) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - CometColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt index 38871726a9..96a0ec392e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt @@ -6,24 +6,24 @@ +- Window (21) +- * CometColumnarToRow (20) +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -38,104 +38,105 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] (23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(25) CometColumnarToRow [codegen id : 3] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== @@ -148,18 +149,18 @@ BroadcastExchange (30) (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct (27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) (28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] (29) CometColumnarToRow [codegen id : 1] @@ -167,6 +168,6 @@ Input [1]: [d_date_sk#14] (30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 58cabe9418..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -5,33 +5,29 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 16 out of 28 eligible operators (57%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt index 788b3f8f4a..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt @@ -1,47 +1,39 @@ -WholeStageCodegen (7) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt index 38871726a9..96a0ec392e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt @@ -6,24 +6,24 @@ +- Window (21) +- * CometColumnarToRow (20) +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * CometColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - +- ReusedExchange (11) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -38,104 +38,105 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(8) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +(11) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] +Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -(14) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometExchange Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometHashAggregate Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(18) CometExchange +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] (21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] (23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] (24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(25) CometColumnarToRow [codegen id : 3] +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== @@ -148,18 +149,18 @@ BroadcastExchange (30) (26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] +Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct (27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) (28) CometProject -Input [2]: [d_date_sk#14, d_date#22] +Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] (29) CometColumnarToRow [codegen id : 1] @@ -167,6 +168,6 @@ Input [1]: [d_date_sk#14] (30) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt index 58cabe9418..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt @@ -5,33 +5,29 @@ CometColumnarToRow +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 16 out of 28 eligible operators (57%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 788b3f8f4a..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt @@ -1,47 +1,39 @@ -WholeStageCodegen (7) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (2) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 34a99ff7f9b407a622bbfef7eb6a578100358be3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 09:06:17 -0700 Subject: [PATCH 07/10] revert a change --- .../org/apache/comet/serde/operator/CometAggregate.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala index 4654b722ee..30a97546a1 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala @@ -44,11 +44,6 @@ trait CometBaseAggregate { val resultExpressions = aggregate.resultExpressions val child = aggregate.child - if (!isCometShuffleEnabled(aggregate.conf)) { - withInfo(aggregate, "Aggregates are only supported when Comet shuffle is enabled") - return None - } - if (groupingExpressions.isEmpty && aggregateExpressions.isEmpty) { withInfo(aggregate, "No group by or aggregation") return None From af0fccf1ed4fad772e2db6451bbcc6e2e3c95ae1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 09:47:28 -0700 Subject: [PATCH 08/10] remove unused import --- .../scala/org/apache/comet/serde/operator/CometAggregate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala index 30a97546a1..93e5d52c8d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometAggregate.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregat import org.apache.spark.sql.types.MapType import org.apache.comet.{CometConf, ConfigEntry} -import org.apache.comet.CometSparkSessionExtensions.{isCometShuffleEnabled, withInfo} +import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto} From 8f2fa902c433c3f33f65d5c32724532ba078fe5c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 18:17:34 -0700 Subject: [PATCH 09/10] fix --- .../apache/comet/serde/QueryPlanSerde.scala | 28 +++++-------------- 1 file changed, 7 insertions(+), 21 deletions(-) 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 3bf605bb38..29992da73d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1011,28 +1011,14 @@ object QueryPlanSerde extends Logging with CometExprShim { None } - case op => - opSerdeMap.get(op.getClass) match { - case Some(handler) => - handler.enabledConfig.foreach { enabledConfig => - if (!enabledConfig.get(op.conf)) { - withInfo( - op, - s"Native support for operator ${op.getClass.getSimpleName} is disabled. " + - s"Set ${enabledConfig.key}=true to enable it.") - return None - } - } - handler.asInstanceOf[CometOperatorSerde[SparkPlan]].convert(op, builder, childOp: _*) - case _ => - // Emit warning if: - // 1. it is not Spark shuffle operator, which is handled separately - // 2. it is not a Comet operator - if (!op.nodeName.contains("Comet") && !op.isInstanceOf[ShuffleExchangeExec]) { - withInfo(op, s"unsupported Spark operator: ${op.nodeName}") - } - None + case _ => + // Emit warning if: + // 1. it is not Spark shuffle operator, which is handled separately + // 2. it is not a Comet operator + if (!op.nodeName.contains("Comet") && !op.isInstanceOf[ShuffleExchangeExec]) { + withInfo(op, s"unsupported Spark operator: ${op.nodeName}") } + None } } From e54e08bc13af67827a76754f17b068211dbcdcc1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 9 Nov 2025 19:07:50 -0700 Subject: [PATCH 10/10] fix --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 29992da73d..edae4453a7 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1015,7 +1015,8 @@ object QueryPlanSerde extends Logging with CometExprShim { // Emit warning if: // 1. it is not Spark shuffle operator, which is handled separately // 2. it is not a Comet operator - if (!op.nodeName.contains("Comet") && !op.isInstanceOf[ShuffleExchangeExec]) { + if (serde.isEmpty && !op.nodeName.contains("Comet") && + !op.isInstanceOf[ShuffleExchangeExec]) { withInfo(op, s"unsupported Spark operator: ${op.nodeName}") } None